You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2016/12/02 13:34:51 UTC

[01/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Repository: flink
Updated Branches:
  refs/heads/master cc006ff18 -> 85a19bd4b


http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java
deleted file mode 100644
index 0effdd8..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.serialization;
-
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * This is a deserialization schema specific for the Flink Kinesis Consumer. Different from the
- * basic {@link DeserializationSchema}, this schema offers additional Kinesis-specific information
- * about the record that may be useful to the user application.
- *
- * @param <T> The type created by the keyed deserialization schema.
- */
-public interface KinesisDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
-
-	/**
-	 * Deserializes a Kinesis record's bytes
-	 *
-	 * @param recordValue the record's value as a byte array
-	 * @param partitionKey the record's partition key at the time of writing
-	 * @param seqNum the sequence number of this record in the Kinesis shard
-	 * @param approxArrivalTimestamp the server-side timestamp of when Kinesis received and stored the record
-	 * @param stream the name of the Kinesis stream that this record was sent to
-	 * @param shardId The identifier of the shard the record was sent to
-	 * @return the deserialized message as an Java object
-	 * @throws IOException
-	 */
-	T deserialize(byte[] recordValue, String partitionKey, String seqNum, long approxArrivalTimestamp, String stream, String shardId) throws IOException;
-
-	/**
-	 * Method to decide whether the element signals the end of the stream. If
-	 * true is returned the element won't be emitted.
-	 *
-	 * @param nextElement the element to test for the end-of-stream signal
-	 * @return true if the element signals end of stream, false otherwise
-	 */
-	// TODO FLINK-4194 ADD SUPPORT FOR boolean isEndOfStream(T nextElement);
-}


[45/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
new file mode 100644
index 0000000..73067c1
--- /dev/null
+++ b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.io.IOException;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableInput;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.reflect.ReflectDatumReader;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.flink.api.common.io.CheckpointableInputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.flink.api.avro.FSDataInputStreamWrapper;
+import org.apache.flink.api.common.io.FileInputFormat;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.util.InstantiationUtil;
+
+/**
+ * Provides a {@link FileInputFormat} for Avro records.
+ *
+ * @param <E>
+ *            the type of the result Avro record. If you specify
+ *            {@link GenericRecord} then the result will be returned as a
+ *            {@link GenericRecord}, so you do not have to know the schema ahead
+ *            of time.
+ */
+public class AvroInputFormat<E> extends FileInputFormat<E> implements ResultTypeQueryable<E>,
+	CheckpointableInputFormat<FileInputSplit, Tuple2<Long, Long>> {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(AvroInputFormat.class);
+
+	private final Class<E> avroValueType;
+	
+	private boolean reuseAvroValue = true;
+
+	private transient DataFileReader<E> dataFileReader;
+
+	private transient long end;
+
+	private transient long recordsReadSinceLastSync;
+
+	private long lastSync = -1l;
+
+	public AvroInputFormat(Path filePath, Class<E> type) {
+		super(filePath);
+		this.avroValueType = type;
+	}
+
+	/**
+	 * Sets the flag whether to reuse the Avro value instance for all records.
+	 * By default, the input format reuses the Avro value.
+	 *
+	 * @param reuseAvroValue True, if the input format should reuse the Avro value instance, false otherwise.
+	 */
+	public void setReuseAvroValue(boolean reuseAvroValue) {
+		this.reuseAvroValue = reuseAvroValue;
+	}
+
+	/**
+	 * If set, the InputFormat will only read entire files.
+	 */
+	public void setUnsplittable(boolean unsplittable) {
+		this.unsplittable = unsplittable;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	// Typing
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public TypeInformation<E> getProducedType() {
+		return TypeExtractor.getForClass(this.avroValueType);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	// Input Format Methods
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void open(FileInputSplit split) throws IOException {
+		super.open(split);
+		dataFileReader = initReader(split);
+		dataFileReader.sync(split.getStart());
+		lastSync = dataFileReader.previousSync();
+	}
+
+	private DataFileReader<E> initReader(FileInputSplit split) throws IOException {
+		DatumReader<E> datumReader;
+
+		if (org.apache.avro.generic.GenericRecord.class == avroValueType) {
+			datumReader = new GenericDatumReader<E>();
+		} else {
+			datumReader = org.apache.avro.specific.SpecificRecordBase.class.isAssignableFrom(avroValueType)
+				? new SpecificDatumReader<E>(avroValueType) : new ReflectDatumReader<E>(avroValueType);
+		}
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Opening split {}", split);
+		}
+
+		SeekableInput in = new FSDataInputStreamWrapper(stream, split.getPath().getFileSystem().getFileStatus(split.getPath()).getLen());
+		DataFileReader<E> dataFileReader = (DataFileReader) DataFileReader.openReader(in, datumReader);
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Loaded SCHEMA: {}", dataFileReader.getSchema());
+		}
+
+		end = split.getStart() + split.getLength();
+		recordsReadSinceLastSync = 0;
+		return dataFileReader;
+	}
+
+	@Override
+	public boolean reachedEnd() throws IOException {
+		return !dataFileReader.hasNext() || dataFileReader.pastSync(end);
+	}
+
+	public long getRecordsReadFromBlock() {
+		return this.recordsReadSinceLastSync;
+	}
+
+	@Override
+	public E nextRecord(E reuseValue) throws IOException {
+		if (reachedEnd()) {
+			return null;
+		}
+
+		// if we start a new block, then register the event, and
+		// restart the counter.
+		if(dataFileReader.previousSync() != lastSync) {
+			lastSync = dataFileReader.previousSync();
+			recordsReadSinceLastSync = 0;
+		}
+		recordsReadSinceLastSync++;
+
+		if (reuseAvroValue) {
+			return dataFileReader.next(reuseValue);
+		} else {
+			if (GenericRecord.class == avroValueType) {
+				return dataFileReader.next();
+			} else {
+				return dataFileReader.next(InstantiationUtil.instantiate(avroValueType, Object.class));
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Checkpointing
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public Tuple2<Long, Long> getCurrentState() throws IOException {
+		return new Tuple2<>(this.lastSync, this.recordsReadSinceLastSync);
+	}
+
+	@Override
+	public void reopen(FileInputSplit split, Tuple2<Long, Long> state) throws IOException {
+		Preconditions.checkNotNull(split, "reopen() cannot be called on a null split.");
+		Preconditions.checkNotNull(state, "reopen() cannot be called with a null initial state.");
+
+		try {
+			this.open(split);
+		} finally {
+			if (state.f0 != -1) {
+				lastSync = state.f0;
+				recordsReadSinceLastSync = state.f1;
+			}
+		}
+
+		if (lastSync != -1) {
+			// open and read until the record we were before
+			// the checkpoint and discard the values
+			dataFileReader.seek(lastSync);
+			for(int i = 0; i < recordsReadSinceLastSync; i++) {
+				dataFileReader.next(null);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
new file mode 100644
index 0000000..600d1e5
--- /dev/null
+++ b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.avro.Schema;
+import org.apache.avro.file.CodecFactory;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.reflect.ReflectDatumWriter;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.flink.api.common.io.FileOutputFormat;
+import org.apache.flink.core.fs.Path;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+public class AvroOutputFormat<E> extends FileOutputFormat<E> implements Serializable {
+
+	/**
+	 * Wrapper which encapsulates the supported codec and a related serialization byte.
+	 */
+	public enum Codec {
+
+		NULL((byte)0, CodecFactory.nullCodec()),
+		SNAPPY((byte)1, CodecFactory.snappyCodec()),
+		BZIP2((byte)2, CodecFactory.bzip2Codec()),
+		DEFLATE((byte)3, CodecFactory.deflateCodec(CodecFactory.DEFAULT_DEFLATE_LEVEL)),
+		XZ((byte)4, CodecFactory.xzCodec(CodecFactory.DEFAULT_XZ_LEVEL));
+
+		private byte codecByte;
+
+		private CodecFactory codecFactory;
+
+		Codec(final byte codecByte, final CodecFactory codecFactory) {
+			this.codecByte = codecByte;
+			this.codecFactory = codecFactory;
+		}
+
+		private byte getCodecByte() {
+			return codecByte;
+		}
+
+		private CodecFactory getCodecFactory() {
+			return codecFactory;
+		}
+
+		private static Codec forCodecByte(byte codecByte) {
+			for (final Codec codec : Codec.values()) {
+				if (codec.getCodecByte() == codecByte) {
+					return codec;
+				}
+			}
+			throw new IllegalArgumentException("no codec for codecByte: " + codecByte);
+		}
+	}
+
+	private static final long serialVersionUID = 1L;
+
+	private final Class<E> avroValueType;
+
+	private transient Schema userDefinedSchema = null;
+
+	private transient Codec codec = null;
+	
+	private transient DataFileWriter<E> dataFileWriter;
+
+	public AvroOutputFormat(Path filePath, Class<E> type) {
+		super(filePath);
+		this.avroValueType = type;
+	}
+
+	public AvroOutputFormat(Class<E> type) {
+		this.avroValueType = type;
+	}
+
+	@Override
+	protected String getDirectoryFileName(int taskNumber) {
+		return super.getDirectoryFileName(taskNumber) + ".avro";
+	}
+
+	public void setSchema(Schema schema) {
+		this.userDefinedSchema = schema;
+	}
+
+	/**
+	 * Set avro codec for compression.
+	 *
+	 * @param codec avro codec.
+	 */
+	public void setCodec(final Codec codec) {
+		this.codec = checkNotNull(codec, "codec can not be null");
+	}
+
+	@Override
+	public void writeRecord(E record) throws IOException {
+		dataFileWriter.append(record);
+	}
+
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		super.open(taskNumber, numTasks);
+
+		DatumWriter<E> datumWriter;
+		Schema schema;
+		if (org.apache.avro.specific.SpecificRecordBase.class.isAssignableFrom(avroValueType)) {
+			datumWriter = new SpecificDatumWriter<E>(avroValueType);
+			try {
+				schema = ((org.apache.avro.specific.SpecificRecordBase)avroValueType.newInstance()).getSchema();
+			} catch (InstantiationException e) {
+				throw new RuntimeException(e.getMessage());
+			} catch (IllegalAccessException e) {
+				throw new RuntimeException(e.getMessage());
+			}
+		} else {
+			datumWriter = new ReflectDatumWriter<E>(avroValueType);
+			schema = ReflectData.get().getSchema(avroValueType);
+		}
+		dataFileWriter = new DataFileWriter<E>(datumWriter);
+		if (codec != null) {
+			dataFileWriter.setCodec(codec.getCodecFactory());
+		}
+		if (userDefinedSchema == null) {
+			dataFileWriter.create(schema, stream);
+		} else {
+			dataFileWriter.create(userDefinedSchema, stream);
+		}
+	}
+
+	private void writeObject(java.io.ObjectOutputStream out) throws IOException {
+		out.defaultWriteObject();
+
+		if (codec != null) {
+			out.writeByte(codec.getCodecByte());
+		} else {
+			out.writeByte(-1);
+		}
+
+		if(userDefinedSchema != null) {
+			byte[] json = userDefinedSchema.toString().getBytes();
+			out.writeInt(json.length);
+			out.write(json);
+		} else {
+			out.writeInt(0);
+		}
+	}
+
+	private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
+		in.defaultReadObject();
+
+		byte codecByte = in.readByte();
+		if (codecByte >= 0) {
+			setCodec(Codec.forCodecByte(codecByte));
+		}
+
+		int length = in.readInt();
+		if(length != 0) {
+			byte[] json = new byte[length];
+			in.readFully(json);
+
+			Schema schema = new Schema.Parser().parse(new String(json));
+			setSchema(schema);
+		}
+	}
+
+	@Override
+	public void close() throws IOException {
+		dataFileWriter.flush();
+		dataFileWriter.close();
+		super.close();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/assembly/test-assembly.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/assembly/test-assembly.xml b/flink-connectors/flink-avro/src/test/assembly/test-assembly.xml
new file mode 100644
index 0000000..0f4561a
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/assembly/test-assembly.xml
@@ -0,0 +1,36 @@
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<assembly>
+	<id>test-jar</id>
+	<formats>
+		<format>jar</format>
+	</formats>
+	<includeBaseDirectory>false</includeBaseDirectory>
+	<fileSets>
+		<fileSet>
+			<directory>${project.build.testOutputDirectory}</directory>
+			<outputDirectory>/</outputDirectory>
+			<!--modify/add include to match your package(s) -->
+			<includes>
+				<include>org/apache/flink/api/avro/testjar/**</include>
+			</includes>
+		</fileSet>
+	</fileSets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
new file mode 100644
index 0000000..1030ff8
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.avro;
+
+import java.io.File;
+
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.PackagedProgram;
+import org.apache.flink.client.program.StandaloneClusterClient;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class AvroExternalJarProgramITCase {
+
+	private static final String JAR_FILE = "maven-test-jar.jar";
+
+	private static final String TEST_DATA_FILE = "/testdata.avro";
+
+	@Test
+	public void testExternalProgram() {
+
+		LocalFlinkMiniCluster testMiniCluster = null;
+
+		try {
+			Configuration config = new Configuration();
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
+			testMiniCluster = new LocalFlinkMiniCluster(config, false);
+			testMiniCluster.start();
+
+			String jarFile = JAR_FILE;
+			String testData = getClass().getResource(TEST_DATA_FILE).toString();
+
+			PackagedProgram program = new PackagedProgram(new File(jarFile), new String[] { testData });
+
+
+			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, testMiniCluster.getLeaderRPCPort());
+
+			ClusterClient client = new StandaloneClusterClient(config);
+
+			client.setPrintStatusDuringExecution(false);
+			client.run(program, 4);
+
+		}
+		catch (Throwable t) {
+			System.err.println(t.getMessage());
+			t.printStackTrace();
+			Assert.fail("Error during the packaged program execution: " + t.getMessage());
+		}
+		finally {
+			if (testMiniCluster != null) {
+				try {
+					testMiniCluster.stop();
+				} catch (Throwable t) {
+					// ignore
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java
new file mode 100644
index 0000000..3b01ccb
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.avro;
+
+import org.junit.Assert;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.reflect.ReflectDatumReader;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.flink.api.io.avro.example.User;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.io.AvroOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+@SuppressWarnings("serial")
+public class AvroOutputFormatITCase extends JavaProgramTestBase {
+
+	public static String outputPath1;
+
+	public static String outputPath2;
+
+	public static String inputPath;
+
+	public static String userData = "alice|1|blue\n" +
+		"bob|2|red\n" +
+		"john|3|yellow\n" +
+		"walt|4|black\n";
+
+	@Override
+	protected void preSubmit() throws Exception {
+		inputPath = createTempFile("user", userData);
+		outputPath1 = getTempDirPath("avro_output1");
+		outputPath2 = getTempDirPath("avro_output2");
+	}
+
+
+	@Override
+	protected void testProgram() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<String, Integer, String>> input = env.readCsvFile(inputPath)
+			.fieldDelimiter("|")
+			.types(String.class, Integer.class, String.class);
+
+		//output the data with AvroOutputFormat for specific user type
+		DataSet<User> specificUser = input.map(new ConvertToUser());
+		AvroOutputFormat<User> avroOutputFormat = new AvroOutputFormat<User>(User.class);
+		avroOutputFormat.setCodec(AvroOutputFormat.Codec.SNAPPY); // FLINK-4771: use a codec
+		avroOutputFormat.setSchema(User.SCHEMA$); //FLINK-3304: Ensure the OF is properly serializing the schema
+		specificUser.write(avroOutputFormat, outputPath1);
+
+		//output the data with AvroOutputFormat for reflect user type
+		DataSet<ReflectiveUser> reflectiveUser = specificUser.map(new ConvertToReflective());
+		reflectiveUser.write(new AvroOutputFormat<ReflectiveUser>(ReflectiveUser.class), outputPath2);
+
+		env.execute();
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		//compare result for specific user type
+		File [] output1;
+		File file1 = asFile(outputPath1);
+		if (file1.isDirectory()) {
+			output1 = file1.listFiles();
+			// check for avro ext in dir.
+			for (File avroOutput : output1) {
+				Assert.assertTrue("Expect extension '.avro'", avroOutput.toString().endsWith(".avro"));
+			}
+		} else {
+			output1 = new File[] {file1};
+		}
+		List<String> result1 = new ArrayList<String>();
+		DatumReader<User> userDatumReader1 = new SpecificDatumReader<User>(User.class);
+		for (File avroOutput : output1) {
+
+			DataFileReader<User> dataFileReader1 = new DataFileReader<User>(avroOutput, userDatumReader1);
+			while (dataFileReader1.hasNext()) {
+				User user = dataFileReader1.next();
+				result1.add(user.getName() + "|" + user.getFavoriteNumber() + "|" + user.getFavoriteColor());
+			}
+		}
+		for (String expectedResult : userData.split("\n")) {
+			Assert.assertTrue("expected user " + expectedResult + " not found.", result1.contains(expectedResult));
+		}
+
+		//compare result for reflect user type
+		File [] output2;
+		File file2 = asFile(outputPath2);
+		if (file2.isDirectory()) {
+			output2 = file2.listFiles();
+		} else {
+			output2 = new File[] {file2};
+		}
+		List<String> result2 = new ArrayList<String>();
+		DatumReader<ReflectiveUser> userDatumReader2 = new ReflectDatumReader<ReflectiveUser>(ReflectiveUser.class);
+		for (File avroOutput : output2) {
+			DataFileReader<ReflectiveUser> dataFileReader2 = new DataFileReader<ReflectiveUser>(avroOutput, userDatumReader2);
+			while (dataFileReader2.hasNext()) {
+				ReflectiveUser user = dataFileReader2.next();
+				result2.add(user.getName() + "|" + user.getFavoriteNumber() + "|" + user.getFavoriteColor());
+			}
+		}
+		for (String expectedResult : userData.split("\n")) {
+			Assert.assertTrue("expected user " + expectedResult + " not found.", result2.contains(expectedResult));
+		}
+
+
+	}
+
+
+	public final static class ConvertToUser extends RichMapFunction<Tuple3<String, Integer, String>, User> {
+
+		@Override
+		public User map(Tuple3<String, Integer, String> value) throws Exception {
+			return new User(value.f0, value.f1, value.f2);
+		}
+	}
+
+	public final static class ConvertToReflective extends RichMapFunction<User, ReflectiveUser> {
+
+		@Override
+		public ReflectiveUser map(User value) throws Exception {
+			return new ReflectiveUser(value.getName().toString(), value.getFavoriteNumber(), value.getFavoriteColor().toString());
+		}
+	}
+
+	
+	public static class ReflectiveUser {
+		private String name;
+		private int favoriteNumber;
+		private String favoriteColor;
+
+		public ReflectiveUser() {}
+
+		public ReflectiveUser(String name, int favoriteNumber, String favoriteColor) {
+			this.name = name;
+			this.favoriteNumber = favoriteNumber;
+			this.favoriteColor = favoriteColor;
+		}
+		
+		public String getName() {
+			return this.name;
+		}
+		public String getFavoriteColor() {
+			return this.favoriteColor;
+		}
+		public int getFavoriteNumber() {
+			return this.favoriteNumber;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/EncoderDecoderTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/EncoderDecoderTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/EncoderDecoderTest.java
new file mode 100644
index 0000000..c39db15
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/EncoderDecoderTest.java
@@ -0,0 +1,528 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.avro;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.avro.reflect.ReflectDatumReader;
+import org.apache.avro.reflect.ReflectDatumWriter;
+import org.apache.flink.api.io.avro.generated.Address;
+import org.apache.flink.api.io.avro.generated.Colors;
+import org.apache.flink.api.io.avro.generated.Fixed16;
+import org.apache.flink.api.io.avro.generated.User;
+import org.apache.flink.util.StringUtils;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests the {@link DataOutputEncoder} and {@link DataInputDecoder} classes for Avro serialization.
+ */
+public class EncoderDecoderTest {
+	@Test
+	public void testComplexStringsDirecty() {
+		try {
+			Random rnd = new Random(349712539451944123L);
+			
+			for (int i = 0; i < 10; i++) {
+				String testString = StringUtils.getRandomString(rnd, 10, 100);
+				
+				ByteArrayOutputStream baos = new ByteArrayOutputStream(512);
+				{
+					DataOutputStream dataOut = new DataOutputStream(baos);
+					DataOutputEncoder encoder = new DataOutputEncoder();
+					encoder.setOut(dataOut);
+					
+					encoder.writeString(testString);
+					dataOut.flush();
+					dataOut.close();
+				}
+				
+				byte[] data = baos.toByteArray();
+				
+				// deserialize
+				{
+					ByteArrayInputStream bais = new ByteArrayInputStream(data);
+					DataInputStream dataIn = new DataInputStream(bais);
+					DataInputDecoder decoder = new DataInputDecoder();
+					decoder.setIn(dataIn);
+	
+					String deserialized = decoder.readString();
+					
+					assertEquals(testString, deserialized);
+				}
+			}
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			fail("Test failed due to an exception: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testPrimitiveTypes() {
+		
+		testObjectSerialization(new Boolean(true));
+		testObjectSerialization(new Boolean(false));
+		
+		testObjectSerialization(Byte.valueOf((byte) 0));
+		testObjectSerialization(Byte.valueOf((byte) 1));
+		testObjectSerialization(Byte.valueOf((byte) -1));
+		testObjectSerialization(Byte.valueOf(Byte.MIN_VALUE));
+		testObjectSerialization(Byte.valueOf(Byte.MAX_VALUE));
+		
+		testObjectSerialization(Short.valueOf((short) 0));
+		testObjectSerialization(Short.valueOf((short) 1));
+		testObjectSerialization(Short.valueOf((short) -1));
+		testObjectSerialization(Short.valueOf(Short.MIN_VALUE));
+		testObjectSerialization(Short.valueOf(Short.MAX_VALUE));
+		
+		testObjectSerialization(Integer.valueOf(0));
+		testObjectSerialization(Integer.valueOf(1));
+		testObjectSerialization(Integer.valueOf(-1));
+		testObjectSerialization(Integer.valueOf(Integer.MIN_VALUE));
+		testObjectSerialization(Integer.valueOf(Integer.MAX_VALUE));
+		
+		testObjectSerialization(Long.valueOf(0));
+		testObjectSerialization(Long.valueOf(1));
+		testObjectSerialization(Long.valueOf(-1));
+		testObjectSerialization(Long.valueOf(Long.MIN_VALUE));
+		testObjectSerialization(Long.valueOf(Long.MAX_VALUE));
+		
+		testObjectSerialization(Float.valueOf(0));
+		testObjectSerialization(Float.valueOf(1));
+		testObjectSerialization(Float.valueOf(-1));
+		testObjectSerialization(Float.valueOf((float)Math.E));
+		testObjectSerialization(Float.valueOf((float)Math.PI));
+		testObjectSerialization(Float.valueOf(Float.MIN_VALUE));
+		testObjectSerialization(Float.valueOf(Float.MAX_VALUE));
+		testObjectSerialization(Float.valueOf(Float.MIN_NORMAL));
+		testObjectSerialization(Float.valueOf(Float.NaN));
+		testObjectSerialization(Float.valueOf(Float.NEGATIVE_INFINITY));
+		testObjectSerialization(Float.valueOf(Float.POSITIVE_INFINITY));
+		
+		testObjectSerialization(Double.valueOf(0));
+		testObjectSerialization(Double.valueOf(1));
+		testObjectSerialization(Double.valueOf(-1));
+		testObjectSerialization(Double.valueOf(Math.E));
+		testObjectSerialization(Double.valueOf(Math.PI));
+		testObjectSerialization(Double.valueOf(Double.MIN_VALUE));
+		testObjectSerialization(Double.valueOf(Double.MAX_VALUE));
+		testObjectSerialization(Double.valueOf(Double.MIN_NORMAL));
+		testObjectSerialization(Double.valueOf(Double.NaN));
+		testObjectSerialization(Double.valueOf(Double.NEGATIVE_INFINITY));
+		testObjectSerialization(Double.valueOf(Double.POSITIVE_INFINITY));
+		
+		testObjectSerialization("");
+		testObjectSerialization("abcdefg");
+		testObjectSerialization("ab\u1535\u0155xyz\u706F");
+		
+		testObjectSerialization(new SimpleTypes(3637, 54876486548L, (byte) 65, "We're out looking for astronauts", (short) 0x2387, 2.65767523));
+		testObjectSerialization(new SimpleTypes(705608724, -1L, (byte) -65, "Serve me the sky with a big slice of lemon", (short) Byte.MIN_VALUE, 0.0000001));
+	}
+	
+	@Test
+	public void testArrayTypes() {
+		{
+			int[] array = new int[] {1, 2, 3, 4, 5};
+			testObjectSerialization(array);
+		}
+		{
+			long[] array = new long[] {1, 2, 3, 4, 5};
+			testObjectSerialization(array);
+		}
+		{
+			float[] array = new float[] {1, 2, 3, 4, 5};
+			testObjectSerialization(array);
+		}
+		{
+			double[] array = new double[] {1, 2, 3, 4, 5};
+			testObjectSerialization(array);
+		}
+		{
+			String[] array = new String[] {"Oh", "my", "what", "do", "we", "have", "here", "?"};
+			testObjectSerialization(array);
+		}
+	}
+	
+	@Test
+	public void testEmptyArray() {
+		{
+			int[] array = new int[0];
+			testObjectSerialization(array);
+		}
+		{
+			long[] array = new long[0];
+			testObjectSerialization(array);
+		}
+		{
+			float[] array = new float[0];
+			testObjectSerialization(array);
+		}
+		{
+			double[] array = new double[0];
+			testObjectSerialization(array);
+		}
+		{
+			String[] array = new String[0];
+			testObjectSerialization(array);
+		}
+	}
+	
+	@Test
+	public void testObjects() {
+		// simple object containing only primitives
+		{
+			testObjectSerialization(new Book(976243875L, "The Serialization Odysse", 42));
+		}
+		
+		// object with collection
+		{
+			ArrayList<String> list = new ArrayList<String>();
+			list.add("A");
+			list.add("B");
+			list.add("C");
+			list.add("D");
+			list.add("E");
+			
+			testObjectSerialization(new BookAuthor(976243875L, list, "Arno Nym"));
+		}
+		
+		// object with empty collection
+		{
+			ArrayList<String> list = new ArrayList<String>();
+			testObjectSerialization(new BookAuthor(987654321L, list, "The Saurus"));
+		}
+	}
+	
+	@Test
+	public void testNestedObjectsWithCollections() {
+		testObjectSerialization(new ComplexNestedObject2(true));
+	}
+	
+	@Test
+	public void testGeneratedObjectWithNullableFields() {
+		List<CharSequence> strings = Arrays.asList(new CharSequence[] { "These", "strings", "should", "be", "recognizable", "as", "a", "meaningful", "sequence" });
+		List<Boolean> bools = Arrays.asList(true, true, false, false, true, false, true, true);
+		Map<CharSequence, Long> map = new HashMap<CharSequence, Long>();
+		map.put("1", 1L);
+		map.put("2", 2L);
+		map.put("3", 3L);
+
+		byte[] b = new byte[16];
+		new Random().nextBytes(b);
+		Fixed16 f = new Fixed16(b);
+		Address addr = new Address(new Integer(239), "6th Main", "Bangalore",
+				"Karnataka", "560075");
+		User user = new User("Freudenreich", 1337, "macintosh gray",
+				1234567890L, 3.1415926, null, true, strings, bools, null,
+				Colors.GREEN, map, f, new Boolean(true), addr);
+		
+		testObjectSerialization(user);
+	}
+	
+	@Test
+	public void testVarLenCountEncoding() {
+		try {
+			long[] values = new long[] { 0, 1, 2, 3, 4, 0, 574, 45236, 0, 234623462, 23462462346L, 0, 9734028767869761L, 0x7fffffffffffffffL};
+			
+			// write
+			ByteArrayOutputStream baos = new ByteArrayOutputStream(512);
+			{
+				DataOutputStream dataOut = new DataOutputStream(baos);
+				
+				for (long val : values) {
+					DataOutputEncoder.writeVarLongCount(dataOut, val);
+				}
+				
+				dataOut.flush();
+				dataOut.close();
+			}
+			
+			// read
+			{
+				ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+				DataInputStream dataIn = new DataInputStream(bais);
+				
+				for (long val : values) {
+					long read = DataInputDecoder.readVarLongCount(dataIn);
+					assertEquals("Wrong var-len encoded value read.", val, read);
+				}
+			}
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			fail("Test failed due to an exception: " + e.getMessage());
+		}
+	}
+	
+	private static <X> void testObjectSerialization(X obj) {
+		
+		try {
+			
+			// serialize
+			ByteArrayOutputStream baos = new ByteArrayOutputStream(512);
+			{
+				DataOutputStream dataOut = new DataOutputStream(baos);
+				DataOutputEncoder encoder = new DataOutputEncoder();
+				encoder.setOut(dataOut);
+				
+				@SuppressWarnings("unchecked")
+				Class<X> clazz = (Class<X>) obj.getClass();
+				ReflectDatumWriter<X> writer = new ReflectDatumWriter<X>(clazz);
+				
+				writer.write(obj, encoder);
+				dataOut.flush();
+				dataOut.close();
+			}
+			
+			byte[] data = baos.toByteArray();
+			X result = null;
+			
+			// deserialize
+			{
+				ByteArrayInputStream bais = new ByteArrayInputStream(data);
+				DataInputStream dataIn = new DataInputStream(bais);
+				DataInputDecoder decoder = new DataInputDecoder();
+				decoder.setIn(dataIn);
+
+				@SuppressWarnings("unchecked")
+				Class<X> clazz = (Class<X>) obj.getClass();
+				ReflectDatumReader<X> reader = new ReflectDatumReader<X>(clazz);
+				
+				// create a reuse object if possible, otherwise we have no reuse object 
+				X reuse = null;
+				try {
+					@SuppressWarnings("unchecked")
+					X test = (X) obj.getClass().newInstance();
+					reuse = test;
+				} catch (Throwable t) {}
+				
+				result = reader.read(reuse, decoder);
+			}
+			
+			// check
+			final String message = "Deserialized object is not the same as the original";
+			
+			if (obj.getClass().isArray()) {
+				Class<?> clazz = obj.getClass();
+				if (clazz == byte[].class) {
+					assertArrayEquals(message, (byte[]) obj, (byte[]) result);
+				}
+				else if (clazz == short[].class) {
+					assertArrayEquals(message, (short[]) obj, (short[]) result);
+				}
+				else if (clazz == int[].class) {
+					assertArrayEquals(message, (int[]) obj, (int[]) result);
+				}
+				else if (clazz == long[].class) {
+					assertArrayEquals(message, (long[]) obj, (long[]) result);
+				}
+				else if (clazz == char[].class) {
+					assertArrayEquals(message, (char[]) obj, (char[]) result);
+				}
+				else if (clazz == float[].class) {
+					assertArrayEquals(message, (float[]) obj, (float[]) result, 0.0f);
+				}
+				else if (clazz == double[].class) {
+					assertArrayEquals(message, (double[]) obj, (double[]) result, 0.0);
+				} else {
+					assertArrayEquals(message, (Object[]) obj, (Object[]) result);
+				}
+			} else {
+				assertEquals(message, obj, result);
+			}
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			fail("Test failed due to an exception: " + e.getMessage());
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Test Objects
+	// --------------------------------------------------------------------------------------------
+
+
+	public static final class SimpleTypes {
+		
+		private final int iVal;
+		private final long lVal;
+		private final byte bVal;
+		private final String sVal;
+		private final short rVal;
+		private final double dVal;
+		
+		
+		public SimpleTypes() {
+			this(0, 0, (byte) 0, "", (short) 0, 0);
+		}
+		
+		public SimpleTypes(int iVal, long lVal, byte bVal, String sVal, short rVal, double dVal) {
+			this.iVal = iVal;
+			this.lVal = lVal;
+			this.bVal = bVal;
+			this.sVal = sVal;
+			this.rVal = rVal;
+			this.dVal = dVal;
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (obj.getClass() == SimpleTypes.class) {
+				SimpleTypes other = (SimpleTypes) obj;
+				
+				return other.iVal == this.iVal &&
+						other.lVal == this.lVal &&
+						other.bVal == this.bVal &&
+						other.sVal.equals(this.sVal) &&
+						other.rVal == this.rVal &&
+						other.dVal == this.dVal;
+				
+			} else {
+				return false;
+			}
+		}
+	}
+	
+	public static class ComplexNestedObject1 {
+		
+		private double doubleValue;
+		
+		private List<String> stringList;
+		
+		public ComplexNestedObject1() {}
+		
+		public ComplexNestedObject1(int offInit) {
+			this.doubleValue = 6293485.6723 + offInit;
+				
+			this.stringList = new ArrayList<String>();
+			this.stringList.add("A" + offInit);
+			this.stringList.add("somewhat" + offInit);
+			this.stringList.add("random" + offInit);
+			this.stringList.add("collection" + offInit);
+			this.stringList.add("of" + offInit);
+			this.stringList.add("strings" + offInit);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (obj.getClass() == ComplexNestedObject1.class) {
+				ComplexNestedObject1 other = (ComplexNestedObject1) obj;
+				return other.doubleValue == this.doubleValue && this.stringList.equals(other.stringList);
+			} else {
+				return false;
+			}
+		}
+	}
+	
+	public static class ComplexNestedObject2 {
+		
+		private long longValue;
+		
+		private Map<String, ComplexNestedObject1> theMap;
+		
+		public ComplexNestedObject2() {}
+		
+		public ComplexNestedObject2(boolean init) {
+			this.longValue = 46547;
+				
+			this.theMap = new HashMap<String, ComplexNestedObject1>();
+			this.theMap.put("36354L", new ComplexNestedObject1(43546543));
+			this.theMap.put("785611L", new ComplexNestedObject1(45784568));
+			this.theMap.put("43L", new ComplexNestedObject1(9876543));
+			this.theMap.put("-45687L", new ComplexNestedObject1(7897615));
+			this.theMap.put("1919876876896L", new ComplexNestedObject1(27154));
+			this.theMap.put("-868468468L", new ComplexNestedObject1(546435));
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (obj.getClass() == ComplexNestedObject2.class) {
+				ComplexNestedObject2 other = (ComplexNestedObject2) obj;
+				return other.longValue == this.longValue && this.theMap.equals(other.theMap);
+			} else {
+				return false;
+			}
+		}
+	}
+	
+	public static class Book {
+
+		private long bookId;
+		private String title;
+		private long authorId;
+
+		public Book() {}
+
+		public Book(long bookId, String title, long authorId) {
+			this.bookId = bookId;
+			this.title = title;
+			this.authorId = authorId;
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (obj.getClass() == Book.class) {
+				Book other = (Book) obj;
+				return other.bookId == this.bookId && other.authorId == this.authorId && this.title.equals(other.title);
+			} else {
+				return false;
+			}
+		}
+	}
+
+	public static class BookAuthor {
+
+		private long authorId;
+		private List<String> bookTitles;
+		private String authorName;
+
+		public BookAuthor() {}
+
+		public BookAuthor(long authorId, List<String> bookTitles, String authorName) {
+			this.authorId = authorId;
+			this.bookTitles = bookTitles;
+			this.authorName = authorName;
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (obj.getClass() == BookAuthor.class) {
+				BookAuthor other = (BookAuthor) obj;
+				return other.authorName.equals(this.authorName) && other.authorId == this.authorId &&
+						other.bookTitles.equals(this.bookTitles);
+			} else {
+				return false;
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
new file mode 100644
index 0000000..1174786
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.avro.testjar;
+
+// ================================================================================================
+//  This file defines the classes for the AvroExternalJarProgramITCase.
+//  The program is exported into src/test/resources/AvroTestProgram.jar.
+//
+//  THIS FILE MUST STAY FULLY COMMENTED SUCH THAT THE HERE DEFINED CLASSES ARE NOT COMPILED
+//  AND ADDED TO THE test-classes DIRECTORY. OTHERWISE, THE EXTERNAL CLASS LOADING WILL
+//  NOT BE COVERED BY THIS TEST.
+// ================================================================================================
+
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.reflect.ReflectDatumWriter;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.AvroInputFormat;
+import org.apache.flink.core.fs.Path;
+
+public class AvroExternalJarProgram  {
+
+	public static final class Color {
+		
+		private String name;
+		private double saturation;
+		
+		public Color() {
+			name = "";
+			saturation = 1.0;
+		}
+		
+		public Color(String name, double saturation) {
+			this.name = name;
+			this.saturation = saturation;
+		}
+		
+		public String getName() {
+			return name;
+		}
+		
+		public void setName(String name) {
+			this.name = name;
+		}
+		
+		public double getSaturation() {
+			return saturation;
+		}
+		
+		public void setSaturation(double saturation) {
+			this.saturation = saturation;
+		}
+		
+		@Override
+		public String toString() {
+			return name + '(' + saturation + ')';
+		}
+	}
+	
+	public static final class MyUser {
+		
+		private String name;
+		private List<Color> colors;
+		
+		public MyUser() {
+			name = "unknown";
+			colors = new ArrayList<Color>();
+		}
+		
+		public MyUser(String name, List<Color> colors) {
+			this.name = name;
+			this.colors = colors;
+		}
+		
+		public String getName() {
+			return name;
+		}
+		
+		public List<Color> getColors() {
+			return colors;
+		}
+		
+		public void setName(String name) {
+			this.name = name;
+		}
+		
+		public void setColors(List<Color> colors) {
+			this.colors = colors;
+		}
+		
+		@Override
+		public String toString() {
+			return name + " : " + colors;
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static final class NameExtractor extends RichMapFunction<MyUser, Tuple2<String, MyUser>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<String, MyUser> map(MyUser u) {
+			String namePrefix = u.getName().substring(0, 1);
+			return new Tuple2<String, MyUser>(namePrefix, u);
+		}
+	}
+	
+	public static final class NameGrouper extends RichReduceFunction<Tuple2<String, MyUser>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<String, MyUser> reduce(Tuple2<String, MyUser> val1, Tuple2<String, MyUser> val2) {
+			return val1;
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Test Data
+	// --------------------------------------------------------------------------------------------
+	
+	public static final class Generator {
+		
+		private final Random rnd = new Random(2389756789345689276L);
+		
+		public MyUser nextUser() {
+			return randomUser();
+		}
+		
+		private MyUser randomUser() {
+			
+			int numColors = rnd.nextInt(5);
+			ArrayList<Color> colors = new ArrayList<Color>(numColors);
+			for (int i = 0; i < numColors; i++) {
+				colors.add(new Color(randomString(), rnd.nextDouble()));
+			}
+			
+			return new MyUser(randomString(), colors);
+		}
+		
+		private String randomString() {
+			char[] c = new char[this.rnd.nextInt(20) + 5];
+			
+			for (int i = 0; i < c.length; i++) {
+				c[i] = (char) (this.rnd.nextInt(150) + 40);
+			}
+			
+			return new String(c);
+		}
+	}
+	
+	public static void writeTestData(File testFile, int numRecords) throws IOException {
+		
+		DatumWriter<MyUser> userDatumWriter = new ReflectDatumWriter<MyUser>(MyUser.class);
+		DataFileWriter<MyUser> dataFileWriter = new DataFileWriter<MyUser>(userDatumWriter);
+		
+		dataFileWriter.create(ReflectData.get().getSchema(MyUser.class), testFile);
+		
+		
+		Generator generator = new Generator();
+		
+		for (int i = 0; i < numRecords; i++) {
+			MyUser user = generator.nextUser();
+			dataFileWriter.append(user);
+		}
+		
+		dataFileWriter.close();
+	}
+
+//	public static void main(String[] args) throws Exception {
+//		String testDataFile = new File("src/test/resources/testdata.avro").getAbsolutePath();
+//		writeTestData(new File(testDataFile), 50);
+//	}
+	
+	public static void main(String[] args) throws Exception {
+		String inputPath = args[0];
+		
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		DataSet<MyUser> input = env.createInput(new AvroInputFormat<MyUser>(new Path(inputPath), MyUser.class));
+	
+		DataSet<Tuple2<String, MyUser>> result = input.map(new NameExtractor()).groupBy(0).reduce(new NameGrouper());
+		
+		result.output(new DiscardingOutputFormat<Tuple2<String,MyUser>>());
+		env.execute();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
new file mode 100644
index 0000000..f33f433
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.io.avro;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.io.avro.generated.User;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.io.AvroInputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+@RunWith(Parameterized.class)
+public class AvroPojoTest extends MultipleProgramsTestBase {
+	public AvroPojoTest(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	private File inFile;
+	private String resultPath;
+	private String expected;
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Before
+	public void before() throws Exception{
+		resultPath = tempFolder.newFile().toURI().toString();
+		inFile = tempFolder.newFile();
+		AvroRecordInputFormatTest.writeTestFile(inFile);
+	}
+
+	@After
+	public void after() throws Exception{
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+
+	@Test
+	public void testSimpleAvroRead() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Path in = new Path(inFile.getAbsoluteFile().toURI());
+
+		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
+		DataSet<User> usersDS = env.createInput(users)
+				// null map type because the order changes in different JVMs (hard to test)
+		.map(new MapFunction<User, User>() {
+			@Override
+			public User map(User value) throws Exception {
+				value.setTypeMap(null);
+				return value;
+			}
+		});
+
+		usersDS.writeAsText(resultPath);
+
+		env.execute("Simple Avro read job");
+
+
+		expected = "{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null, \"type_long_test\": null, \"type_double_test\": 123.45, \"type_null_test\": null, \"type_bool_test\": true, \"type_array_string\": [\"ELEMENT 1\", \"ELEMENT 2\"], \"type_array_boolean\": [true, false], \"type_nullable_array\": null, \"type_enum\": \"GREEN\", \"type_map\": null, \"type_fixed\": null, \"type_union\": null, \"type_nested\": {\"num\": 239, \"street\": \"Baker Street\", \"city\": \"London\", \"state\": \"London\", \"zip\": \"NW1 6XE\"}}\n" +
+					"{\"name\": \"Charlie\", \"favorite_number\": null, \"favorite_color\": \"blue\", \"type_long_test\": 1337, \"type_double_test\": 1.337, \"type_null_test\": null, \"type_bool_test\": false, \"type_array_string\": [], \"type_array_boolean\": [], \"type_nullable_array\": null, \"type_enum\": \"RED\", \"type_map\": null, \"type_fixed\": null, \"type_union\": null, \"type_nested\": {\"num\": 239, \"street\": \"Baker Street\", \"city\": \"London\", \"state\": \"London\", \"zip\": \"NW1 6XE\"}}\n";
+	}
+
+	@Test
+	public void testSerializeWithAvro() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().enableForceAvro();
+		Path in = new Path(inFile.getAbsoluteFile().toURI());
+
+		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
+		DataSet<User> usersDS = env.createInput(users)
+				// null map type because the order changes in different JVMs (hard to test)
+				.map(new MapFunction<User, User>() {
+					@Override
+					public User map(User value) throws Exception {
+						Map<CharSequence, Long> ab = new HashMap<CharSequence, Long>(1);
+						ab.put("hehe", 12L);
+						value.setTypeMap(ab);
+						return value;
+					}
+				});
+
+		usersDS.writeAsText(resultPath);
+
+		env.execute("Simple Avro read job");
+
+
+		expected = "{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null, \"type_long_test\": null, \"type_double_test\": 123.45, \"type_null_test\": null, \"type_bool_test\": true, \"type_array_string\": [\"ELEMENT 1\", \"ELEMENT 2\"], \"type_array_boolean\": [true, false], \"type_nullable_array\": null, \"type_enum\": \"GREEN\", \"type_map\": {\"hehe\": 12}, \"type_fixed\": null, \"type_union\": null, \"type_nested\": {\"num\": 239, \"street\": \"Baker Street\", \"city\": \"London\", \"state\": \"London\", \"zip\": \"NW1 6XE\"}}\n" +
+					"{\"name\": \"Charlie\", \"favorite_number\": null, \"favorite_color\": \"blue\", \"type_long_test\": 1337, \"type_double_test\": 1.337, \"type_null_test\": null, \"type_bool_test\": false, \"type_array_string\": [], \"type_array_boolean\": [], \"type_nullable_array\": null, \"type_enum\": \"RED\", \"type_map\": {\"hehe\": 12}, \"type_fixed\": null, \"type_union\": null, \"type_nested\": {\"num\": 239, \"street\": \"Baker Street\", \"city\": \"London\", \"state\": \"London\", \"zip\": \"NW1 6XE\"}}\n";
+
+	}
+
+	@Test
+	public void testKeySelection() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().enableObjectReuse();
+		Path in = new Path(inFile.getAbsoluteFile().toURI());
+
+		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
+		DataSet<User> usersDS = env.createInput(users);
+
+		DataSet<Tuple2<String, Integer>> res = usersDS.groupBy("name").reduceGroup(new GroupReduceFunction<User, Tuple2<String, Integer>>() {
+			@Override
+			public void reduce(Iterable<User> values, Collector<Tuple2<String, Integer>> out) throws Exception {
+				for (User u : values) {
+					out.collect(new Tuple2<String, Integer>(u.getName().toString(), 1));
+				}
+			}
+		});
+		res.writeAsText(resultPath);
+		env.execute("Avro Key selection");
+
+
+		expected = "(Alyssa,1)\n(Charlie,1)\n";
+	}
+
+	@Test
+	public void testWithAvroGenericSer() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().enableForceAvro();
+		Path in = new Path(inFile.getAbsoluteFile().toURI());
+
+		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
+		DataSet<User> usersDS = env.createInput(users);
+
+		DataSet<Tuple2<String, Integer>> res = usersDS.groupBy(new KeySelector<User, String>() {
+			@Override
+			public String getKey(User value) throws Exception {
+				return String.valueOf(value.getName());
+			}
+		}).reduceGroup(new GroupReduceFunction<User, Tuple2<String, Integer>>() {
+			@Override
+			public void reduce(Iterable<User> values, Collector<Tuple2<String, Integer>> out) throws Exception {
+				for(User u : values) {
+					out.collect(new Tuple2<String, Integer>(u.getName().toString(), 1));
+				}
+			}
+		});
+
+		res.writeAsText(resultPath);
+		env.execute("Avro Key selection");
+
+
+		expected = "(Charlie,1)\n(Alyssa,1)\n";
+	}
+
+	@Test
+	public void testWithKryoGenericSer() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().enableForceKryo();
+		Path in = new Path(inFile.getAbsoluteFile().toURI());
+
+		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
+		DataSet<User> usersDS = env.createInput(users);
+
+		DataSet<Tuple2<String, Integer>> res = usersDS.groupBy(new KeySelector<User, String>() {
+			@Override
+			public String getKey(User value) throws Exception {
+				return String.valueOf(value.getName());
+			}
+		}).reduceGroup(new GroupReduceFunction<User, Tuple2<String, Integer>>() {
+			@Override
+			public void reduce(Iterable<User> values, Collector<Tuple2<String, Integer>> out) throws Exception {
+				for (User u : values) {
+					out.collect(new Tuple2<String, Integer>(u.getName().toString(), 1));
+				}
+			}
+		});
+
+		res.writeAsText(resultPath);
+		env.execute("Avro Key selection");
+
+
+		expected = "(Charlie,1)\n(Alyssa,1)\n";
+	}
+
+	/**
+	 * Test some know fields for grouping on
+	 */
+	@Test
+	public void testAllFields() throws Exception {
+		for(String fieldName : Arrays.asList("name", "type_enum", "type_double_test")) {
+			testField(fieldName);
+		}
+	}
+
+	private void testField(final String fieldName) throws Exception {
+		before();
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Path in = new Path(inFile.getAbsoluteFile().toURI());
+
+		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
+		DataSet<User> usersDS = env.createInput(users);
+
+		DataSet<Object> res = usersDS.groupBy(fieldName).reduceGroup(new GroupReduceFunction<User, Object>() {
+			@Override
+			public void reduce(Iterable<User> values, Collector<Object> out) throws Exception {
+				for(User u : values) {
+					out.collect(u.get(fieldName));
+				}
+			}
+		});
+		res.writeAsText(resultPath);
+		env.execute("Simple Avro read job");
+
+		// test if automatic registration of the Types worked
+		ExecutionConfig ec = env.getConfig();
+		Assert.assertTrue(ec.getRegisteredKryoTypes().contains(org.apache.flink.api.io.avro.generated.Fixed16.class));
+
+		if(fieldName.equals("name")) {
+			expected = "Alyssa\nCharlie";
+		} else if(fieldName.equals("type_enum")) {
+			expected = "GREEN\nRED\n";
+		} else if(fieldName.equals("type_double_test")) {
+			expected = "123.45\n1.337\n";
+		} else {
+			Assert.fail("Unknown field");
+		}
+
+		after();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
new file mode 100644
index 0000000..91a9612
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.avro;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.file.FileReader;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.io.avro.generated.Address;
+import org.apache.flink.api.io.avro.generated.Colors;
+import org.apache.flink.api.io.avro.generated.User;
+import org.apache.flink.api.java.io.AvroInputFormat;
+import org.apache.flink.api.java.typeutils.AvroTypeInfo;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the avro input format.
+ * (The testcase is mostly the getting started tutorial of avro)
+ * http://avro.apache.org/docs/current/gettingstartedjava.html
+ */
+public class AvroRecordInputFormatTest {
+	
+	public File testFile;
+	
+	final static String TEST_NAME = "Alyssa";
+	
+	final static String TEST_ARRAY_STRING_1 = "ELEMENT 1";
+	final static String TEST_ARRAY_STRING_2 = "ELEMENT 2";
+	
+	final static boolean TEST_ARRAY_BOOLEAN_1 = true;
+	final static boolean TEST_ARRAY_BOOLEAN_2 = false;
+	
+	final static Colors TEST_ENUM_COLOR = Colors.GREEN;
+	
+	final static String TEST_MAP_KEY1 = "KEY 1";
+	final static long TEST_MAP_VALUE1 = 8546456L;
+	final static String TEST_MAP_KEY2 = "KEY 2";
+	final static long TEST_MAP_VALUE2 = 17554L;
+	
+	final static int TEST_NUM = 239;
+	final static String TEST_STREET = "Baker Street";
+	final static String TEST_CITY = "London";
+	final static String TEST_STATE = "London";
+	final static String TEST_ZIP = "NW1 6XE";
+	
+
+	private Schema userSchema = new User().getSchema();
+
+
+	public static void writeTestFile(File testFile) throws IOException {
+		ArrayList<CharSequence> stringArray = new ArrayList<CharSequence>();
+		stringArray.add(TEST_ARRAY_STRING_1);
+		stringArray.add(TEST_ARRAY_STRING_2);
+
+		ArrayList<Boolean> booleanArray = new ArrayList<Boolean>();
+		booleanArray.add(TEST_ARRAY_BOOLEAN_1);
+		booleanArray.add(TEST_ARRAY_BOOLEAN_2);
+
+		HashMap<CharSequence, Long> longMap = new HashMap<CharSequence, Long>();
+		longMap.put(TEST_MAP_KEY1, TEST_MAP_VALUE1);
+		longMap.put(TEST_MAP_KEY2, TEST_MAP_VALUE2);
+		
+		Address addr = new Address();
+		addr.setNum(TEST_NUM);
+		addr.setStreet(TEST_STREET);
+		addr.setCity(TEST_CITY);
+		addr.setState(TEST_STATE);
+		addr.setZip(TEST_ZIP);
+
+
+		User user1 = new User();
+
+		user1.setName(TEST_NAME);
+		user1.setFavoriteNumber(256);
+		user1.setTypeDoubleTest(123.45d);
+		user1.setTypeBoolTest(true);
+		user1.setTypeArrayString(stringArray);
+		user1.setTypeArrayBoolean(booleanArray);
+		user1.setTypeEnum(TEST_ENUM_COLOR);
+		user1.setTypeMap(longMap);
+		user1.setTypeNested(addr);
+
+		// Construct via builder
+		User user2 = User.newBuilder()
+				.setName("Charlie")
+				.setFavoriteColor("blue")
+				.setFavoriteNumber(null)
+				.setTypeBoolTest(false)
+				.setTypeDoubleTest(1.337d)
+				.setTypeNullTest(null)
+				.setTypeLongTest(1337L)
+				.setTypeArrayString(new ArrayList<CharSequence>())
+				.setTypeArrayBoolean(new ArrayList<Boolean>())
+				.setTypeNullableArray(null)
+				.setTypeEnum(Colors.RED)
+				.setTypeMap(new HashMap<CharSequence, Long>())
+				.setTypeFixed(null)
+				.setTypeUnion(null)
+				.setTypeNested(
+						Address.newBuilder().setNum(TEST_NUM).setStreet(TEST_STREET)
+								.setCity(TEST_CITY).setState(TEST_STATE).setZip(TEST_ZIP)
+								.build())
+				.build();
+		DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
+		DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
+		dataFileWriter.create(user1.getSchema(), testFile);
+		dataFileWriter.append(user1);
+		dataFileWriter.append(user2);
+		dataFileWriter.close();
+	}
+	@Before
+	public void createFiles() throws IOException {
+		testFile = File.createTempFile("AvroInputFormatTest", null);
+		writeTestFile(testFile);
+	}
+
+
+	/**
+	 * Test if the AvroInputFormat is able to properly read data from an avro file.
+	 * @throws IOException
+	 */
+	@Test
+	public void testDeserialisation() throws IOException {
+		Configuration parameters = new Configuration();
+		
+		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
+		
+		format.configure(parameters);
+		FileInputSplit[] splits = format.createInputSplits(1);
+		assertEquals(splits.length, 1);
+		format.open(splits[0]);
+		
+		User u = format.nextRecord(null);
+		assertNotNull(u);
+		
+		String name = u.getName().toString();
+		assertNotNull("empty record", name);
+		assertEquals("name not equal", TEST_NAME, name);
+		
+		// check arrays
+		List<CharSequence> sl = u.getTypeArrayString();
+		assertEquals("element 0 not equal", TEST_ARRAY_STRING_1, sl.get(0).toString());
+		assertEquals("element 1 not equal", TEST_ARRAY_STRING_2, sl.get(1).toString());
+		
+		List<Boolean> bl = u.getTypeArrayBoolean();
+		assertEquals("element 0 not equal", TEST_ARRAY_BOOLEAN_1, bl.get(0));
+		assertEquals("element 1 not equal", TEST_ARRAY_BOOLEAN_2, bl.get(1));
+		
+		// check enums
+		Colors enumValue = u.getTypeEnum();
+		assertEquals("enum not equal", TEST_ENUM_COLOR, enumValue);
+		
+		// check maps
+		Map<CharSequence, Long> lm = u.getTypeMap();
+		assertEquals("map value of key 1 not equal", TEST_MAP_VALUE1, lm.get(new Utf8(TEST_MAP_KEY1)).longValue());
+		assertEquals("map value of key 2 not equal", TEST_MAP_VALUE2, lm.get(new Utf8(TEST_MAP_KEY2)).longValue());
+		
+		assertFalse("expecting second element", format.reachedEnd());
+		assertNotNull("expecting second element", format.nextRecord(u));
+		
+		assertNull(format.nextRecord(u));
+		assertTrue(format.reachedEnd());
+		
+		format.close();
+	}
+
+	/**
+	 * Test if the AvroInputFormat is able to properly read data from an avro file.
+	 * @throws IOException
+	 */
+	@Test
+	public void testDeserialisationReuseAvroRecordFalse() throws IOException {
+		Configuration parameters = new Configuration();
+		
+		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
+		format.setReuseAvroValue(false);
+		
+		format.configure(parameters);
+		FileInputSplit[] splits = format.createInputSplits(1);
+		assertEquals(splits.length, 1);
+		format.open(splits[0]);
+		
+		User u = format.nextRecord(null);
+		assertNotNull(u);
+		
+		String name = u.getName().toString();
+		assertNotNull("empty record", name);
+		assertEquals("name not equal", TEST_NAME, name);
+		
+		// check arrays
+		List<CharSequence> sl = u.getTypeArrayString();
+		assertEquals("element 0 not equal", TEST_ARRAY_STRING_1, sl.get(0).toString());
+		assertEquals("element 1 not equal", TEST_ARRAY_STRING_2, sl.get(1).toString());
+		
+		List<Boolean> bl = u.getTypeArrayBoolean();
+		assertEquals("element 0 not equal", TEST_ARRAY_BOOLEAN_1, bl.get(0));
+		assertEquals("element 1 not equal", TEST_ARRAY_BOOLEAN_2, bl.get(1));
+		
+		// check enums
+		Colors enumValue = u.getTypeEnum();
+		assertEquals("enum not equal", TEST_ENUM_COLOR, enumValue);
+		
+		// check maps
+		Map<CharSequence, Long> lm = u.getTypeMap();
+		assertEquals("map value of key 1 not equal", TEST_MAP_VALUE1, lm.get(new Utf8(TEST_MAP_KEY1)).longValue());
+		assertEquals("map value of key 2 not equal", TEST_MAP_VALUE2, lm.get(new Utf8(TEST_MAP_KEY2)).longValue());
+		
+		assertFalse("expecting second element", format.reachedEnd());
+		assertNotNull("expecting second element", format.nextRecord(u));
+		
+		assertNull(format.nextRecord(u));
+		assertTrue(format.reachedEnd());
+		
+		format.close();
+	}
+
+	/**
+	 * Test if the Flink serialization is able to properly process GenericData.Record types.
+	 * Usually users of Avro generate classes (POJOs) from Avro schemas.
+	 * However, if generated classes are not available, one can also use GenericData.Record.
+	 * It is an untyped key-value record which is using a schema to validate the correctness of the data.
+	 *
+	 * It is not recommended to use GenericData.Record with Flink. Use generated POJOs instead.
+	 */
+	@Test
+	public void testDeserializeToGenericType() throws IOException {
+		DatumReader<GenericData.Record> datumReader = new GenericDatumReader<>(userSchema);
+
+		try (FileReader<GenericData.Record> dataFileReader = DataFileReader.openReader(testFile, datumReader)) {
+			// initialize Record by reading it from disk (thats easier than creating it by hand)
+			GenericData.Record rec = new GenericData.Record(userSchema);
+			dataFileReader.next(rec);
+			
+			// check if record has been read correctly
+			assertNotNull(rec);
+			assertEquals("name not equal", TEST_NAME, rec.get("name").toString());
+			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), rec.get("type_enum").toString());
+			assertEquals(null, rec.get("type_long_test")); // it is null for the first record.
+
+			// now serialize it with our framework:
+			TypeInformation<GenericData.Record> te = TypeExtractor.createTypeInfo(GenericData.Record.class);
+
+			ExecutionConfig ec = new ExecutionConfig();
+			Assert.assertEquals(GenericTypeInfo.class, te.getClass());
+			
+			Serializers.recursivelyRegisterType(te.getTypeClass(), ec, new HashSet<Class<?>>());
+
+			TypeSerializer<GenericData.Record> tser = te.createSerializer(ec);
+			Assert.assertEquals(1, ec.getDefaultKryoSerializerClasses().size());
+			Assert.assertTrue(
+					ec.getDefaultKryoSerializerClasses().containsKey(Schema.class) &&
+							ec.getDefaultKryoSerializerClasses().get(Schema.class).equals(Serializers.AvroSchemaSerializer.class));
+
+			ByteArrayOutputStream out = new ByteArrayOutputStream();
+			try (DataOutputViewStreamWrapper outView = new DataOutputViewStreamWrapper(out)) {
+				tser.serialize(rec, outView);
+			}
+
+			GenericData.Record newRec;
+			try (DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(
+					new ByteArrayInputStream(out.toByteArray())))
+			{
+				newRec = tser.deserialize(inView);
+			}
+
+			// check if it is still the same
+			assertNotNull(newRec);
+			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), newRec.get("type_enum").toString());
+			assertEquals("name not equal", TEST_NAME, newRec.get("name").toString());
+			assertEquals(null, newRec.get("type_long_test"));
+		}
+	}
+		
+	/**
+	 * This test validates proper serialization with specific (generated POJO) types.
+	 */
+	@Test
+	public void testDeserializeToSpecificType() throws IOException {
+
+		DatumReader<User> datumReader = new SpecificDatumReader<User>(userSchema);
+
+		try (FileReader<User> dataFileReader = DataFileReader.openReader(testFile, datumReader)) {
+			User rec = dataFileReader.next();
+
+			// check if record has been read correctly
+			assertNotNull(rec);
+			assertEquals("name not equal", TEST_NAME, rec.get("name").toString());
+			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), rec.get("type_enum").toString());
+
+			// now serialize it with our framework:
+			ExecutionConfig ec = new ExecutionConfig();
+			TypeInformation<User> te = TypeExtractor.createTypeInfo(User.class);
+
+			Assert.assertEquals(AvroTypeInfo.class, te.getClass());
+			TypeSerializer<User> tser = te.createSerializer(ec);
+
+			ByteArrayOutputStream out = new ByteArrayOutputStream();
+			try (DataOutputViewStreamWrapper outView = new DataOutputViewStreamWrapper(out)) {
+				tser.serialize(rec, outView);
+			}
+
+			User newRec;
+			try (DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(
+					new ByteArrayInputStream(out.toByteArray())))
+			{
+				newRec = tser.deserialize(inView);
+			}
+
+			// check if it is still the same
+			assertNotNull(newRec);
+			assertEquals("name not equal", TEST_NAME, newRec.getName().toString());
+			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), newRec.getTypeEnum().toString());
+		}
+	}
+
+	/**
+	 * Test if the AvroInputFormat is able to properly read data from an Avro
+	 * file as a GenericRecord.
+	 * 
+	 * @throws IOException,
+	 *             if there is an exception
+	 */
+	@Test
+	public void testDeserialisationGenericRecord() throws IOException {
+		Configuration parameters = new Configuration();
+
+		AvroInputFormat<GenericRecord> format = new AvroInputFormat<GenericRecord>(new Path(testFile.getAbsolutePath()),
+				GenericRecord.class);
+
+		doTestDeserializationGenericRecord(format, parameters);
+	}
+
+	/**
+	 * Helper method to test GenericRecord serialisation
+	 * 
+	 * @param format
+	 *            the format to test
+	 * @param parameters
+	 *            the configuration to use
+	 * @throws IOException
+	 *             thrown id there is a issue
+	 */
+	@SuppressWarnings("unchecked")
+	private void doTestDeserializationGenericRecord(final AvroInputFormat<GenericRecord> format,
+			final Configuration parameters) throws IOException {
+		try {
+			format.configure(parameters);
+			FileInputSplit[] splits = format.createInputSplits(1);
+			assertEquals(splits.length, 1);
+			format.open(splits[0]);
+
+			GenericRecord u = format.nextRecord(null);
+			assertNotNull(u);
+			assertEquals("The schemas should be equal", userSchema, u.getSchema());
+
+			String name = u.get("name").toString();
+			assertNotNull("empty record", name);
+			assertEquals("name not equal", TEST_NAME, name);
+
+			// check arrays
+			List<CharSequence> sl = (List<CharSequence>) u.get("type_array_string");
+			assertEquals("element 0 not equal", TEST_ARRAY_STRING_1, sl.get(0).toString());
+			assertEquals("element 1 not equal", TEST_ARRAY_STRING_2, sl.get(1).toString());
+
+			List<Boolean> bl = (List<Boolean>) u.get("type_array_boolean");
+			assertEquals("element 0 not equal", TEST_ARRAY_BOOLEAN_1, bl.get(0));
+			assertEquals("element 1 not equal", TEST_ARRAY_BOOLEAN_2, bl.get(1));
+
+			// check enums
+			GenericData.EnumSymbol enumValue = (GenericData.EnumSymbol) u.get("type_enum");
+			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), enumValue.toString());
+
+			// check maps
+			Map<CharSequence, Long> lm = (Map<CharSequence, Long>) u.get("type_map");
+			assertEquals("map value of key 1 not equal", TEST_MAP_VALUE1, lm.get(new Utf8(TEST_MAP_KEY1)).longValue());
+			assertEquals("map value of key 2 not equal", TEST_MAP_VALUE2, lm.get(new Utf8(TEST_MAP_KEY2)).longValue());
+
+			assertFalse("expecting second element", format.reachedEnd());
+			assertNotNull("expecting second element", format.nextRecord(u));
+
+			assertNull(format.nextRecord(u));
+			assertTrue(format.reachedEnd());
+		} finally {
+			format.close();
+		}
+	}
+
+	/**
+	 * Test if the AvroInputFormat is able to properly read data from an avro
+	 * file as a GenericRecord
+	 * 
+	 * @throws IOException,
+	 *             if there is an error
+	 */
+	@Test
+	public void testDeserialisationGenericRecordReuseAvroValueFalse() throws IOException {
+		Configuration parameters = new Configuration();
+
+		AvroInputFormat<GenericRecord> format = new AvroInputFormat<GenericRecord>(new Path(testFile.getAbsolutePath()),
+				GenericRecord.class);
+		format.configure(parameters);
+		format.setReuseAvroValue(false);
+
+		doTestDeserializationGenericRecord(format, parameters);
+	}
+
+	@After
+	public void deleteFiles() {
+		testFile.delete();
+	}
+
+}


[05/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index aa7ea49..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ /dev/null
@@ -1,2006 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import kafka.consumer.Consumer;
-import kafka.consumer.ConsumerConfig;
-import kafka.consumer.ConsumerIterator;
-import kafka.consumer.KafkaStream;
-import kafka.javaapi.consumer.ConsumerConnector;
-import kafka.message.MessageAndMetadata;
-import kafka.server.KafkaServer;
-import org.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;
-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.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.runtime.client.JobCancellationException;
-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.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;
-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.testutils.DataGenerators;
-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;
-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.KeyedDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.test.util.SuccessException;
-import org.apache.flink.testutils.junit.RetryOnException;
-import org.apache.flink.testutils.junit.RetryRule;
-import org.apache.flink.util.Collector;
-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 javax.management.MBeanServer;
-import javax.management.ObjectName;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.flink.test.util.TestUtils.tryExecute;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-
-@SuppressWarnings("serial")
-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
-	//
-	//  The tests here are all not activated (by an @Test tag), but need
-	//  to be invoked from the extending classes. That way, the classes can
-	//  select which tests to run.
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Test that ensures the KafkaConsumer is properly failing if the topic doesnt exist
-	 * and a wrong broker was specified
-	 *
-	 * @throws Exception
-	 */
-	public void runFailOnNoBrokerTest() throws Exception {
-		try {
-			Properties properties = new Properties();
-
-			StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			see.getConfig().disableSysoutLogging();
-			see.setRestartStrategy(RestartStrategies.noRestart());
-			see.setParallelism(1);
-
-			// use wrong ports for the consumers
-			properties.setProperty("bootstrap.servers", "localhost:80");
-			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");
-			properties.putAll(secureProps);
-			FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer("doesntexist", new SimpleStringSchema(), properties);
-			DataStream<String> stream = see.addSource(source);
-			stream.print();
-			see.execute("No broker test");
-		} catch(ProgramInvocationException pie) {
-			if(kafkaServer.getVersion().equals("0.9") || kafkaServer.getVersion().equals("0.10")) {
-				assertTrue(pie.getCause() instanceof JobExecutionException);
-
-				JobExecutionException jee = (JobExecutionException) pie.getCause();
-
-				assertTrue(jee.getCause() instanceof TimeoutException);
-
-				TimeoutException te = (TimeoutException) jee.getCause();
-
-				assertEquals("Timeout expired while fetching topic metadata", te.getMessage());
-			} else {
-				assertTrue(pie.getCause() instanceof JobExecutionException);
-
-				JobExecutionException jee = (JobExecutionException) pie.getCause();
-
-				assertTrue(jee.getCause() instanceof RuntimeException);
-
-				RuntimeException re = (RuntimeException) jee.getCause();
-
-				assertTrue(re.getMessage().contains("Unable to retrieve any partitions for the requested topics [doesntexist]"));
-			}
-		}
-	}
-
-	/**
-	 * Ensures that the committed offsets to Kafka are the offsets of "the next record to process"
-	 */
-	public void runCommitOffsetsToKafka() throws Exception {
-		// 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50)
-		final int parallelism = 3;
-		final int recordsInEachPartition = 50;
-
-		final String topicName = writeSequence("testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1);
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.getConfig().disableSysoutLogging();
-		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env.setParallelism(parallelism);
-		env.enableCheckpointing(200);
-
-		DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps));
-		stream.addSink(new DiscardingSink<String>());
-
-		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
-		final Thread runner = new Thread("runner") {
-			@Override
-			public void run() {
-				try {
-					env.execute();
-				}
-				catch (Throwable t) {
-					if (!(t.getCause() instanceof JobCancellationException)) {
-						errorRef.set(t);
-					}
-				}
-			}
-		};
-		runner.start();
-
-		final Long l50 = 50L; // the final committed offset in Kafka should be 50
-		final long deadline = 30000 + System.currentTimeMillis();
-
-		KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
-
-		do {
-			Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
-			Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
-			Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
-
-			if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3)) {
-				break;
-			}
-
-			Thread.sleep(100);
-		}
-		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);
-		}
-
-		// final check to see if offsets are correctly in Kafka
-		Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
-		Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
-		Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
-		Assert.assertEquals(Long.valueOf(50L), o1);
-		Assert.assertEquals(Long.valueOf(50L), o2);
-		Assert.assertEquals(Long.valueOf(50L), o3);
-
-		kafkaOffsetHandler.close();
-		deleteTestTopic(topicName);
-	}
-
-	/**
-	 * This test first writes a total of 300 records to a test topic, reads the first 150 so that some offsets are
-	 * committed to Kafka, and then startup the consumer again to read the remaining records starting from the committed offsets.
-	 * The test ensures that whatever offsets were committed to Kafka, the consumer correctly picks them up
-	 * and starts at the correct position.
-	 */
-	public void runStartFromKafkaCommitOffsets() throws Exception {
-		final int parallelism = 3;
-		final int recordsInEachPartition = 300;
-
-		final String topicName = writeSequence("testStartFromKafkaCommitOffsetsTopic", recordsInEachPartition, parallelism, 1);
-
-		KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
-
-		Long o1;
-		Long o2;
-		Long o3;
-		int attempt = 0;
-		// make sure that o1, o2, o3 are not all null before proceeding
-		do {
-			attempt++;
-			LOG.info("Attempt " + attempt + " to read records and commit some offsets to Kafka");
-
-			final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env.getConfig().disableSysoutLogging();
-			env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-			env.setParallelism(parallelism);
-			env.enableCheckpointing(20); // fast checkpoints to make sure we commit some offsets
-
-			env
-				.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps))
-				.map(new ThrottledMapper<String>(50))
-				.map(new MapFunction<String, Object>() {
-					int count = 0;
-					@Override
-					public Object map(String value) throws Exception {
-						count++;
-						if (count == 150) {
-							throw new SuccessException();
-						}
-						return null;
-					}
-				})
-				.addSink(new DiscardingSink<>());
-
-			tryExecute(env, "Read some records to commit offsets to Kafka");
-
-			o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
-			o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
-			o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
-		} while (o1 == null && o2 == null && o3 == null && attempt < 3);
-
-		if (o1 == null && o2 == null && o3 == null) {
-			throw new RuntimeException("No offsets have been committed after 3 attempts");
-		}
-
-		LOG.info("Got final committed offsets from Kafka o1={}, o2={}, o3={}", o1, o2, o3);
-
-		final StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env2.getConfig().disableSysoutLogging();
-		env2.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env2.setParallelism(parallelism);
-
-		// whatever offsets were committed for each partition, the consumer should pick
-		// them up and start from the correct position so that the remaining records are all read
-		HashMap<Integer, Tuple2<Integer, Integer>> partitionsToValuesCountAndStartOffset = new HashMap<>();
-		partitionsToValuesCountAndStartOffset.put(0, new Tuple2<>(
-			(o1 != null) ? (int) (recordsInEachPartition - o1) : recordsInEachPartition,
-			(o1 != null) ? o1.intValue() : 0
-		));
-		partitionsToValuesCountAndStartOffset.put(1, new Tuple2<>(
-			(o2 != null) ? (int) (recordsInEachPartition - o2) : recordsInEachPartition,
-			(o2 != null) ? o2.intValue() : 0
-		));
-		partitionsToValuesCountAndStartOffset.put(2, new Tuple2<>(
-			(o3 != null) ? (int) (recordsInEachPartition - o3) : recordsInEachPartition,
-			(o3 != null) ? o3.intValue() : 0
-		));
-
-		readSequence(env2, standardProps, topicName, partitionsToValuesCountAndStartOffset);
-
-		kafkaOffsetHandler.close();
-		deleteTestTopic(topicName);
-	}
-
-	/**
-	 * This test ensures that when the consumers retrieve some start offset from kafka (earliest, latest), that this offset
-	 * is committed to Kafka, even if some partitions are not read.
-	 *
-	 * Test:
-	 * - Create 3 partitions
-	 * - write 50 messages into each.
-	 * - Start three consumers with auto.offset.reset='latest' and wait until they committed into Kafka.
-	 * - Check if the offsets in Kafka are set to 50 for the three partitions
-	 *
-	 * See FLINK-3440 as well
-	 */
-	public void runAutoOffsetRetrievalAndCommitToKafka() throws Exception {
-		// 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50)
-		final int parallelism = 3;
-		final int recordsInEachPartition = 50;
-
-		final String topicName = writeSequence("testAutoOffsetRetrievalAndCommitToKafkaTopic", recordsInEachPartition, parallelism, 1);
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.getConfig().disableSysoutLogging();
-		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env.setParallelism(parallelism);
-		env.enableCheckpointing(200);
-
-		Properties readProps = new Properties();
-		readProps.putAll(standardProps);
-		readProps.setProperty("auto.offset.reset", "latest"); // set to reset to latest, so that partitions are initially not read
-
-		DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), readProps));
-		stream.addSink(new DiscardingSink<String>());
-
-		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
-		final Thread runner = new Thread("runner") {
-			@Override
-			public void run() {
-				try {
-					env.execute();
-				}
-				catch (Throwable t) {
-					if (!(t.getCause() instanceof JobCancellationException)) {
-						errorRef.set(t);
-					}
-				}
-			}
-		};
-		runner.start();
-
-		KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
-
-		final Long l50 = 50L; // the final committed offset in Kafka should be 50
-		final long deadline = 30000 + System.currentTimeMillis();
-		do {
-			Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
-			Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
-			Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
-
-			if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3)) {
-				break;
-			}
-
-			Thread.sleep(100);
-		}
-		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);
-		}
-
-		// final check to see if offsets are correctly in Kafka
-		Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
-		Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
-		Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
-		Assert.assertEquals(Long.valueOf(50L), o1);
-		Assert.assertEquals(Long.valueOf(50L), o2);
-		Assert.assertEquals(Long.valueOf(50L), o3);
-
-		kafkaOffsetHandler.close();
-		deleteTestTopic(topicName);
-	}
-	
-	/**
-	 * Ensure Kafka is working on both producer and consumer side.
-	 * This executes a job that contains two Flink pipelines.
-	 *
-	 * <pre>
-	 * (generator source) --> (kafka sink)-[KAFKA-TOPIC]-(kafka source) --> (validating sink)
-	 * </pre>
-	 * 
-	 * We need to externally retry this test. We cannot let Flink's retry mechanism do it, because the Kafka producer
-	 * does not guarantee exactly-once output. Hence a recovery would introduce duplicates that
-	 * cause the test to fail.
-	 *
-	 * This test also ensures that FLINK-3156 doesn't happen again:
-	 *
-	 * The following situation caused a NPE in the FlinkKafkaConsumer
-	 *
-	 * topic-1 <-- elements are only produced into topic1.
-	 * topic-2
-	 *
-	 * Therefore, this test is consuming as well from an empty topic.
-	 *
-	 */
-	@RetryOnException(times=2, exception=kafka.common.NotLeaderForPartitionException.class)
-	public void runSimpleConcurrentProducerConsumerTopology() throws Exception {
-		final String topic = "concurrentProducerConsumerTopic_" + UUID.randomUUID().toString();
-		final String additionalEmptyTopic = "additionalEmptyTopic_" + UUID.randomUUID().toString();
-
-		final int parallelism = 3;
-		final int elementsPerPartition = 100;
-		final int totalElements = parallelism * elementsPerPartition;
-
-		createTestTopic(topic, parallelism, 2);
-		createTestTopic(additionalEmptyTopic, parallelism, 1); // create an empty topic which will remain empty all the time
-
-		final StreamExecutionEnvironment env =
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(parallelism);
-		env.enableCheckpointing(500);
-		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
-		env.getConfig().disableSysoutLogging();
-
-		TypeInformation<Tuple2<Long, String>> longStringType = TypeInfoParser.parse("Tuple2<Long, String>");
-
-		TypeInformationSerializationSchema<Tuple2<Long, String>> sourceSchema =
-				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
-
-		TypeInformationSerializationSchema<Tuple2<Long, String>> sinkSchema =
-				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
-
-		// ----------- add producer dataflow ----------
-
-		DataStream<Tuple2<Long, String>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Long,String>>() {
-
-			private boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Long, String>> ctx) throws InterruptedException {
-				int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition;
-				int limit = cnt + elementsPerPartition;
-
-
-				while (running && cnt < limit) {
-					ctx.collect(new Tuple2<>(1000L + cnt, "kafka-" + cnt));
-					cnt++;
-					// we delay data generation a bit so that we are sure that some checkpoints are
-					// triggered (for FLINK-3156)
-					Thread.sleep(50);
-				}
-			}
-
-			@Override
-			public void cancel() {
-				running = false;
-			}
-		});
-		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
-		producerProperties.setProperty("retries", "3");
-		producerProperties.putAll(secureProps);
-		kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null);
-
-		// ----------- add consumer dataflow ----------
-
-		List<String> topics = new ArrayList<>();
-		topics.add(topic);
-		topics.add(additionalEmptyTopic);
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-		FlinkKafkaConsumerBase<Tuple2<Long, String>> source = kafkaServer.getConsumer(topics, sourceSchema, props);
-
-		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(source).setParallelism(parallelism);
-
-		consuming.addSink(new RichSinkFunction<Tuple2<Long, String>>() {
-
-			private int elCnt = 0;
-			private BitSet validator = new BitSet(totalElements);
-
-			@Override
-			public void invoke(Tuple2<Long, String> value) throws Exception {
-				String[] sp = value.f1.split("-");
-				int v = Integer.parseInt(sp[1]);
-
-				assertEquals(value.f0 - 1000, (long) v);
-
-				assertFalse("Received tuple twice", validator.get(v));
-				validator.set(v);
-				elCnt++;
-
-				if (elCnt == totalElements) {
-					// check if everything in the bitset is set to true
-					int nc;
-					if ((nc = validator.nextClearBit(0)) != totalElements) {
-						fail("The bitset was not set to 1 on all elements. Next clear:"
-								+ nc + " Set: " + validator);
-					}
-					throw new SuccessException();
-				}
-			}
-
-			@Override
-			public void close() throws Exception {
-				super.close();
-			}
-		}).setParallelism(1);
-
-		try {
-			tryExecutePropagateExceptions(env, "runSimpleConcurrentProducerConsumerTopology");
-		}
-		catch (ProgramInvocationException | JobExecutionException e) {
-			// look for NotLeaderForPartitionException
-			Throwable cause = e.getCause();
-
-			// search for nested SuccessExceptions
-			int depth = 0;
-			while (cause != null && depth++ < 20) {
-				if (cause instanceof kafka.common.NotLeaderForPartitionException) {
-					throw (Exception) cause;
-				}
-				cause = cause.getCause();
-			}
-			throw e;
-		}
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests the proper consumption when having a 1:1 correspondence between kafka partitions and
-	 * Flink sources.
-	 */
-	public void runOneToOneExactlyOnceTest() throws Exception {
-
-		final String topic = "oneToOneTopic";
-		final int parallelism = 5;
-		final int numElementsPerPartition = 1000;
-		final int totalElements = parallelism * numElementsPerPartition;
-		final int failAfterElements = numElementsPerPartition / 3;
-
-		createTestTopic(topic, parallelism, 1);
-
-		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-				kafkaServer,
-				topic, parallelism, numElementsPerPartition, true);
-
-		// run the topology that fails and recovers
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.enableCheckpointing(500);
-		env.setParallelism(parallelism);
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
-		env.getConfig().disableSysoutLogging();
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-
-		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
-
-		env
-				.addSource(kafkaSource)
-				.map(new PartitionValidatingMapper(parallelism, 1))
-				.map(new FailingIdentityMapper<Integer>(failAfterElements))
-				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
-
-		FailingIdentityMapper.failedBefore = false;
-		tryExecute(env, "One-to-one exactly once test");
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests the proper consumption when having fewer Flink sources than Kafka partitions, so
-	 * one Flink source will read multiple Kafka partitions.
-	 */
-	public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception {
-		final String topic = "oneToManyTopic";
-		final int numPartitions = 5;
-		final int numElementsPerPartition = 1000;
-		final int totalElements = numPartitions * numElementsPerPartition;
-		final int failAfterElements = numElementsPerPartition / 3;
-
-		final int parallelism = 2;
-
-		createTestTopic(topic, numPartitions, 1);
-
-		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-				kafkaServer,
-				topic, numPartitions, numElementsPerPartition, false);
-
-		// run the topology that fails and recovers
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.enableCheckpointing(500);
-		env.setParallelism(parallelism);
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
-		env.getConfig().disableSysoutLogging();
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
-
-		env
-				.addSource(kafkaSource)
-				.map(new PartitionValidatingMapper(numPartitions, 3))
-				.map(new FailingIdentityMapper<Integer>(failAfterElements))
-				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
-
-		FailingIdentityMapper.failedBefore = false;
-		tryExecute(env, "One-source-multi-partitions exactly once test");
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests the proper consumption when having more Flink sources than Kafka partitions, which means
-	 * that some Flink sources will read no partitions.
-	 */
-	public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception {
-		final String topic = "manyToOneTopic";
-		final int numPartitions = 5;
-		final int numElementsPerPartition = 1000;
-		final int totalElements = numPartitions * numElementsPerPartition;
-		final int failAfterElements = numElementsPerPartition / 3;
-
-		final int parallelism = 8;
-
-		createTestTopic(topic, numPartitions, 1);
-
-		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-				kafkaServer,
-				topic, numPartitions, numElementsPerPartition, true);
-
-		// run the topology that fails and recovers
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.enableCheckpointing(500);
-		env.setParallelism(parallelism);
-		// set the number of restarts to one. The failing mapper will fail once, then it's only success exceptions.
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
-		env.getConfig().disableSysoutLogging();
-		env.setBufferTimeout(0);
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
-
-		env
-			.addSource(kafkaSource)
-			.map(new PartitionValidatingMapper(numPartitions, 1))
-			.map(new FailingIdentityMapper<Integer>(failAfterElements))
-			.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
-
-		FailingIdentityMapper.failedBefore = false;
-		tryExecute(env, "multi-source-one-partitions exactly once test");
-
-
-		deleteTestTopic(topic);
-	}
-	
-	
-	/**
-	 * Tests that the source can be properly canceled when reading full partitions. 
-	 */
-	public void runCancelingOnFullInputTest() throws Exception {
-		final String topic = "cancelingOnFullTopic";
-
-		final int parallelism = 3;
-		createTestTopic(topic, parallelism, 1);
-
-		// launch a producer thread
-		DataGenerators.InfiniteStringsGenerator generator =
-				new DataGenerators.InfiniteStringsGenerator(kafkaServer, topic);
-		generator.start();
-
-		// launch a consumer asynchronously
-
-		final AtomicReference<Throwable> jobError = new AtomicReference<>();
-
-		final Runnable jobRunner = new Runnable() {
-			@Override
-			public void run() {
-				try {
-					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-					env.setParallelism(parallelism);
-					env.enableCheckpointing(100);
-					env.getConfig().disableSysoutLogging();
-
-					Properties props = new Properties();
-					props.putAll(standardProps);
-					props.putAll(secureProps);
-					FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), props);
-
-					env.addSource(source).addSink(new DiscardingSink<String>());
-
-					env.execute("Runner for CancelingOnFullInputTest");
-				}
-				catch (Throwable t) {
-					jobError.set(t);
-				}
-			}
-		};
-
-		Thread runnerThread = new Thread(jobRunner, "program runner thread");
-		runnerThread.start();
-
-		// wait a bit before canceling
-		Thread.sleep(2000);
-
-		Throwable failueCause = jobError.get();
-		if(failueCause != null) {
-			failueCause.printStackTrace();
-			Assert.fail("Test failed prematurely with: " + failueCause.getMessage());
-		}
-
-		// cancel
-		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout), "Runner for CancelingOnFullInputTest");
-
-		// wait for the program to be done and validate that we failed with the right exception
-		runnerThread.join();
-
-		failueCause = jobError.get();
-		assertNotNull("program did not fail properly due to canceling", failueCause);
-		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
-
-		if (generator.isAlive()) {
-			generator.shutdown();
-			generator.join();
-		}
-		else {
-			Throwable t = generator.getError();
-			if (t != null) {
-				t.printStackTrace();
-				fail("Generator failed: " + t.getMessage());
-			} else {
-				fail("Generator failed with no exception");
-			}
-		}
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests that the source can be properly canceled when reading empty partitions. 
-	 */
-	public void runCancelingOnEmptyInputTest() throws Exception {
-		final String topic = "cancelingOnEmptyInputTopic";
-
-		final int parallelism = 3;
-		createTestTopic(topic, parallelism, 1);
-
-		final AtomicReference<Throwable> error = new AtomicReference<>();
-
-		final Runnable jobRunner = new Runnable() {
-			@Override
-			public void run() {
-				try {
-					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-					env.setParallelism(parallelism);
-					env.enableCheckpointing(100);
-					env.getConfig().disableSysoutLogging();
-
-					Properties props = new Properties();
-					props.putAll(standardProps);
-					props.putAll(secureProps);
-					FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), props);
-
-					env.addSource(source).addSink(new DiscardingSink<String>());
-
-					env.execute("CancelingOnEmptyInputTest");
-				}
-				catch (Throwable t) {
-					LOG.error("Job Runner failed with exception", t);
-					error.set(t);
-				}
-			}
-		};
-
-		Thread runnerThread = new Thread(jobRunner, "program runner thread");
-		runnerThread.start();
-
-		// wait a bit before canceling
-		Thread.sleep(2000);
-
-		Throwable failueCause = error.get();
-		if (failueCause != null) {
-			failueCause.printStackTrace();
-			Assert.fail("Test failed prematurely with: " + failueCause.getMessage());
-		}
-		// cancel
-		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
-
-		// wait for the program to be done and validate that we failed with the right exception
-		runnerThread.join();
-
-		failueCause = error.get();
-		assertNotNull("program did not fail properly due to canceling", failueCause);
-		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests that the source can be properly canceled when reading full partitions. 
-	 */
-	public void runFailOnDeployTest() throws Exception {
-		final String topic = "failOnDeployTopic";
-
-		createTestTopic(topic, 2, 1);
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(12); // needs to be more that the mini cluster has slots
-		env.getConfig().disableSysoutLogging();
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
-
-		env
-				.addSource(kafkaSource)
-				.addSink(new DiscardingSink<Integer>());
-
-		try {
-			env.execute("test fail on deploy");
-			fail("this test should fail with an exception");
-		}
-		catch (ProgramInvocationException e) {
-
-			// validate that we failed due to a NoResourceAvailableException
-			Throwable cause = e.getCause();
-			int depth = 0;
-			boolean foundResourceException = false;
-
-			while (cause != null && depth++ < 20) {
-				if (cause instanceof NoResourceAvailableException) {
-					foundResourceException = true;
-					break;
-				}
-				cause = cause.getCause();
-			}
-
-			assertTrue("Wrong exception", foundResourceException);
-		}
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Test producing and consuming into multiple topics
-	 * @throws java.lang.Exception
-	 */
-	public void runProduceConsumeMultipleTopics() throws java.lang.Exception {
-		final int NUM_TOPICS = 5;
-		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++) {
-			final String topic = "topic-" + i;
-			topics.add(topic);
-			// create topic
-			createTestTopic(topic, i + 1 /*partitions*/, 1);
-		}
-		// run first job, producing into all topics
-		DataStream<Tuple3<Integer, Integer, String>> stream = env.addSource(new RichParallelSourceFunction<Tuple3<Integer, Integer, String>>() {
-
-			@Override
-			public void run(SourceContext<Tuple3<Integer, Integer, String>> ctx) throws Exception {
-				int partition = getRuntimeContext().getIndexOfThisSubtask();
-
-				for (int topicId = 0; topicId < NUM_TOPICS; topicId++) {
-					for (int i = 0; i < NUM_ELEMENTS; i++) {
-						ctx.collect(new Tuple3<>(partition, i, "topic-" + topicId));
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-			}
-		});
-
-		Tuple2WithTopicSchema schema = new Tuple2WithTopicSchema(env.getConfig());
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-		kafkaServer.produceIntoKafka(stream, "dummy", schema, props, null);
-
-		env.execute("Write to topics");
-
-		// run second job consuming from multiple topics
-		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.getConfig().disableSysoutLogging();
-
-		stream = env.addSource(kafkaServer.getConsumer(topics, schema, props));
-
-		stream.flatMap(new FlatMapFunction<Tuple3<Integer, Integer, String>, Integer>() {
-			Map<String, Integer> countPerTopic = new HashMap<>(NUM_TOPICS);
-			@Override
-			public void flatMap(Tuple3<Integer, Integer, String> value, Collector<Integer> out) throws Exception {
-				Integer count = countPerTopic.get(value.f2);
-				if (count == null) {
-					count = 1;
-				} else {
-					count++;
-				}
-				countPerTopic.put(value.f2, count);
-
-				// check map:
-				for (Map.Entry<String, Integer> el: countPerTopic.entrySet()) {
-					if (el.getValue() < NUM_ELEMENTS) {
-						break; // not enough yet
-					}
-					if (el.getValue() > NUM_ELEMENTS) {
-						throw new RuntimeException("There is a failure in the test. I've read " +
-								el.getValue() + " from topic " + el.getKey());
-					}
-				}
-				// we've seen messages from all topics
-				throw new SuccessException();
-			}
-		}).setParallelism(1);
-
-		tryExecute(env, "Count elements from the topics");
-
-
-		// delete all topics again
-		for (int i = 0; i < NUM_TOPICS; i++) {
-			final String topic = "topic-" + i;
-			deleteTestTopic(topic);
-		}
-	}
-
-	/**
-	 * Serialization scheme forwarding byte[] records.
-	 */
-	private static class ByteArraySerializationSchema implements KeyedSerializationSchema<byte[]> {
-
-		@Override
-		public byte[] serializeKey(byte[] element) {
-			return null;
-		}
-
-		@Override
-		public byte[] serializeValue(byte[] element) {
-			return element;
-		}
-
-		@Override
-		public String getTargetTopic(byte[] element) {
-			return null;
-		}
-	}
-
-	private static class Tuple2WithTopicSchema implements KeyedDeserializationSchema<Tuple3<Integer, Integer, String>>,
-			KeyedSerializationSchema<Tuple3<Integer, Integer, String>> {
-
-		private final TypeSerializer<Tuple2<Integer, Integer>> ts;
-		
-		public Tuple2WithTopicSchema(ExecutionConfig ec) {
-			ts = TypeInfoParser.<Tuple2<Integer, Integer>>parse("Tuple2<Integer, Integer>").createSerializer(ec);
-		}
-
-		@Override
-		public Tuple3<Integer, Integer, String> deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
-			DataInputView in = new DataInputViewStreamWrapper(new ByteArrayInputStream(message));
-			Tuple2<Integer, Integer> t2 = ts.deserialize(in);
-			return new Tuple3<>(t2.f0, t2.f1, topic);
-		}
-
-		@Override
-		public boolean isEndOfStream(Tuple3<Integer, Integer, String> nextElement) {
-			return false;
-		}
-
-		@Override
-		public TypeInformation<Tuple3<Integer, Integer, String>> getProducedType() {
-			return TypeInfoParser.parse("Tuple3<Integer, Integer, String>");
-		}
-
-		@Override
-		public byte[] serializeKey(Tuple3<Integer, Integer, String> element) {
-			return null;
-		}
-
-		@Override
-		public byte[] serializeValue(Tuple3<Integer, Integer, String> element) {
-			ByteArrayOutputStream by = new ByteArrayOutputStream();
-			DataOutputView out = new DataOutputViewStreamWrapper(by);
-			try {
-				ts.serialize(new Tuple2<>(element.f0, element.f1), out);
-			} catch (IOException e) {
-				throw new RuntimeException("Error" ,e);
-			}
-			return by.toByteArray();
-		}
-
-		@Override
-		public String getTargetTopic(Tuple3<Integer, Integer, String> element) {
-			return element.f2;
-		}
-	}
-
-	/**
-	 * Test Flink's Kafka integration also with very big records (30MB)
-	 * see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message
-	 *
-	 */
-	public void runBigRecordTestTopology() throws Exception {
-
-		final String topic = "bigRecordTestTopic";
-		final int parallelism = 1; // otherwise, the kafka mini clusters may run out of heap space
-
-		createTestTopic(topic, parallelism, 1);
-
-		final TypeInformation<Tuple2<Long, byte[]>> longBytesInfo = TypeInfoParser.parse("Tuple2<Long, byte[]>");
-
-		final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema =
-				new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setRestartStrategy(RestartStrategies.noRestart());
-		env.getConfig().disableSysoutLogging();
-		env.enableCheckpointing(100);
-		env.setParallelism(parallelism);
-
-		// add consuming topology:
-		Properties consumerProps = new Properties();
-		consumerProps.putAll(standardProps);
-		consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 14));
-		consumerProps.setProperty("max.partition.fetch.bytes", Integer.toString(1024 * 1024 * 14)); // for the new fetcher
-		consumerProps.setProperty("queued.max.message.chunks", "1");
-		consumerProps.putAll(secureProps);
-
-		FlinkKafkaConsumerBase<Tuple2<Long, byte[]>> source = kafkaServer.getConsumer(topic, serSchema, consumerProps);
-		DataStreamSource<Tuple2<Long, byte[]>> consuming = env.addSource(source);
-
-		consuming.addSink(new SinkFunction<Tuple2<Long, byte[]>>() {
-
-			private int elCnt = 0;
-
-			@Override
-			public void invoke(Tuple2<Long, byte[]> value) throws Exception {
-				elCnt++;
-				if (value.f0 == -1) {
-					// we should have seen 11 elements now.
-					if (elCnt == 11) {
-						throw new SuccessException();
-					} else {
-						throw new RuntimeException("There have been "+elCnt+" elements");
-					}
-				}
-				if (elCnt > 10) {
-					throw new RuntimeException("More than 10 elements seen: "+elCnt);
-				}
-			}
-		});
-
-		// add producing topology
-		Properties producerProps = new Properties();
-		producerProps.setProperty("max.request.size", Integer.toString(1024 * 1024 * 15));
-		producerProps.setProperty("retries", "3");
-		producerProps.putAll(secureProps);
-		producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerConnectionStrings);
-
-		DataStream<Tuple2<Long, byte[]>> stream = env.addSource(new RichSourceFunction<Tuple2<Long, byte[]>>() {
-
-			private boolean running;
-
-			@Override
-			public void open(Configuration parameters) throws Exception {
-				super.open(parameters);
-				running = true;
-			}
-
-			@Override
-			public void run(SourceContext<Tuple2<Long, byte[]>> ctx) throws Exception {
-				Random rnd = new Random();
-				long cnt = 0;
-				int sevenMb = 1024 * 1024 * 7;
-
-				while (running) {
-					byte[] wl = new byte[sevenMb + rnd.nextInt(sevenMb)];
-					ctx.collect(new Tuple2<>(cnt++, wl));
-
-					Thread.sleep(100);
-
-					if (cnt == 10) {
-						// signal end
-						ctx.collect(new Tuple2<>(-1L, new byte[]{1}));
-						break;
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-				running = false;
-			}
-		});
-
-		kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(serSchema), producerProps, null);
-
-		tryExecute(env, "big topology test");
-		deleteTestTopic(topic);
-	}
-
-	
-	public void runBrokerFailureTest() throws Exception {
-		final String topic = "brokerFailureTestTopic";
-
-		final int parallelism = 2;
-		final int numElementsPerPartition = 1000;
-		final int totalElements = parallelism * numElementsPerPartition;
-		final int failAfterElements = numElementsPerPartition / 3;
-
-
-		createTestTopic(topic, parallelism, 2);
-
-		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-				kafkaServer,
-				topic, parallelism, numElementsPerPartition, true);
-
-		// find leader to shut down
-		int leaderId = kafkaServer.getLeaderToShutDown(topic);
-
-		LOG.info("Leader to shutdown {}", leaderId);
-
-
-		// run the topology (the consumers must handle the failures)
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(parallelism);
-		env.enableCheckpointing(500);
-		env.setRestartStrategy(RestartStrategies.noRestart());
-		env.getConfig().disableSysoutLogging();
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
-
-		env
-				.addSource(kafkaSource)
-				.map(new PartitionValidatingMapper(parallelism, 1))
-				.map(new BrokerKillingMapper<Integer>(leaderId, failAfterElements))
-				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
-
-		BrokerKillingMapper.killedLeaderBefore = false;
-		tryExecute(env, "Broker failure once test");
-
-		// start a new broker:
-		kafkaServer.restartBroker(leaderId);
-	}
-
-	public void runKeyValueTest() throws Exception {
-		final String topic = "keyvaluetest";
-		createTestTopic(topic, 1, 1);
-		final int ELEMENT_COUNT = 5000;
-
-		// ----------- Write some data into Kafka -------------------
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(1);
-		env.setRestartStrategy(RestartStrategies.noRestart());
-		env.getConfig().disableSysoutLogging();
-
-		DataStream<Tuple2<Long, PojoValue>> kvStream = env.addSource(new SourceFunction<Tuple2<Long, PojoValue>>() {
-			@Override
-			public void run(SourceContext<Tuple2<Long, PojoValue>> ctx) throws Exception {
-				Random rnd = new Random(1337);
-				for (long i = 0; i < ELEMENT_COUNT; i++) {
-					PojoValue pojo = new PojoValue();
-					pojo.when = new Date(rnd.nextLong());
-					pojo.lon = rnd.nextLong();
-					pojo.lat = i;
-					// make every second key null to ensure proper "null" serialization
-					Long key = (i % 2 == 0) ? null : i;
-					ctx.collect(new Tuple2<>(key, pojo));
-				}
-			}
-			@Override
-			public void cancel() {
-			}
-		});
-
-		KeyedSerializationSchema<Tuple2<Long, PojoValue>> schema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig());
-		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
-		producerProperties.setProperty("retries", "3");
-		kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null);
-		env.execute("Write KV to Kafka");
-
-		// ----------- Read the data again -------------------
-
-		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(1);
-		env.setRestartStrategy(RestartStrategies.noRestart());
-		env.getConfig().disableSysoutLogging();
-
-
-		KeyedDeserializationSchema<Tuple2<Long, PojoValue>> readSchema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig());
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-		DataStream<Tuple2<Long, PojoValue>> fromKafka = env.addSource(kafkaServer.getConsumer(topic, readSchema, props));
-		fromKafka.flatMap(new RichFlatMapFunction<Tuple2<Long,PojoValue>, Object>() {
-			long counter = 0;
-			@Override
-			public void flatMap(Tuple2<Long, PojoValue> value, Collector<Object> out) throws Exception {
-				// the elements should be in order.
-				Assert.assertTrue("Wrong value " + value.f1.lat, value.f1.lat == counter );
-				if (value.f1.lat % 2 == 0) {
-					assertNull("key was not null", value.f0);
-				} else {
-					Assert.assertTrue("Wrong value " + value.f0, value.f0 == counter);
-				}
-				counter++;
-				if (counter == ELEMENT_COUNT) {
-					// we got the right number of elements
-					throw new SuccessException();
-				}
-			}
-		});
-
-		tryExecute(env, "Read KV from Kafka");
-
-		deleteTestTopic(topic);
-	}
-
-	public static class PojoValue {
-		public Date when;
-		public long lon;
-		public long lat;
-		public PojoValue() {}
-	}
-
-
-	/**
-	 * Test delete behavior and metrics for producer
-	 * @throws Exception
-	 */
-	public void runAllDeletesTest() throws Exception {
-		final String topic = "alldeletestest";
-		createTestTopic(topic, 1, 1);
-		final int ELEMENT_COUNT = 300;
-
-		// ----------- Write some data into Kafka -------------------
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(1);
-		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env.getConfig().disableSysoutLogging();
-
-		DataStream<Tuple2<byte[], PojoValue>> kvStream = env.addSource(new SourceFunction<Tuple2<byte[], PojoValue>>() {
-			@Override
-			public void run(SourceContext<Tuple2<byte[], PojoValue>> ctx) throws Exception {
-				Random rnd = new Random(1337);
-				for (long i = 0; i < ELEMENT_COUNT; i++) {
-					final byte[] key = new byte[200];
-					rnd.nextBytes(key);
-					ctx.collect(new Tuple2<>(key, (PojoValue) null));
-				}
-			}
-			@Override
-			public void cancel() {
-			}
-		});
-
-		TypeInformationKeyValueSerializationSchema<byte[], PojoValue> schema = new TypeInformationKeyValueSerializationSchema<>(byte[].class, PojoValue.class, env.getConfig());
-
-		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
-		producerProperties.setProperty("retries", "3");
-		producerProperties.putAll(secureProps);
-		kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null);
-
-		env.execute("Write deletes to Kafka");
-
-		// ----------- Read the data again -------------------
-
-		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(1);
-		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env.getConfig().disableSysoutLogging();
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-		DataStream<Tuple2<byte[], PojoValue>> fromKafka = env.addSource(kafkaServer.getConsumer(topic, schema, props));
-
-		fromKafka.flatMap(new RichFlatMapFunction<Tuple2<byte[], PojoValue>, Object>() {
-			long counter = 0;
-			@Override
-			public void flatMap(Tuple2<byte[], PojoValue> value, Collector<Object> out) throws Exception {
-				// ensure that deleted messages are passed as nulls
-				assertNull(value.f1);
-				counter++;
-				if (counter == ELEMENT_COUNT) {
-					// we got the right number of elements
-					throw new SuccessException();
-				}
-			}
-		});
-
-		tryExecute(env, "Read deletes from Kafka");
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Test that ensures that DeserializationSchema.isEndOfStream() is properly evaluated.
-	 *
-	 * @throws Exception
-	 */
-	public void runEndOfStreamTest() throws Exception {
-
-		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);
-		env1.setParallelism(1);
-		env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env1.getConfig().disableSysoutLogging();
-
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-
-		DataStream<Tuple2<Integer, Integer>> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(ELEMENT_COUNT), props));
-		fromKafka.flatMap(new FlatMapFunction<Tuple2<Integer,Integer>, Void>() {
-			@Override
-			public void flatMap(Tuple2<Integer, Integer> value, Collector<Void> out) throws Exception {
-				// noop ;)
-			}
-		});
-
-		JobExecutionResult result = tryExecute(env1, "Consume " + ELEMENT_COUNT + " elements from Kafka");
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Test metrics reporting for consumer
-	 *
-	 * @throws Exception
-	 */
-	public void runMetricsTest() throws Throwable {
-
-		// create a stream with 5 topics
-		final String topic = "metricsStream";
-		createTestTopic(topic, 5, 1);
-
-		final Tuple1<Throwable> error = new Tuple1<>(null);
-		Runnable job = new Runnable() {
-			@Override
-			public void run() {
-				try {
-					// start job writing & reading data.
-					final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-					env1.setParallelism(1);
-					env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-					env1.getConfig().disableSysoutLogging();
-					env1.disableOperatorChaining(); // let the source read everything into the network buffers
-
-					Properties props = new Properties();
-					props.putAll(standardProps);
-					props.putAll(secureProps);
-
-					TypeInformationSerializationSchema<Tuple2<Integer, Integer>> schema = new TypeInformationSerializationSchema<>(TypeInfoParser.<Tuple2<Integer, Integer>>parse("Tuple2<Integer, Integer>"), env1.getConfig());
-					DataStream<Tuple2<Integer, Integer>> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, schema, standardProps));
-					fromKafka.flatMap(new FlatMapFunction<Tuple2<Integer, Integer>, Void>() {
-						@Override
-						public void flatMap(Tuple2<Integer, Integer> value, Collector<Void> out) throws Exception {// no op
-						}
-					});
-
-					DataStream<Tuple2<Integer, Integer>> fromGen = env1.addSource(new RichSourceFunction<Tuple2<Integer, Integer>>() {
-						boolean running = true;
-
-						@Override
-						public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
-							int i = 0;
-							while (running) {
-								ctx.collect(Tuple2.of(i++, getRuntimeContext().getIndexOfThisSubtask()));
-								Thread.sleep(1);
-							}
-						}
-
-						@Override
-						public void cancel() {
-							running = false;
-						}
-					});
-
-					kafkaServer.produceIntoKafka(fromGen, topic, new KeyedSerializationSchemaWrapper<>(schema), standardProps, null);
-
-					env1.execute("Metrics test job");
-				} catch(Throwable t) {
-					LOG.warn("Got exception during execution", t);
-					if(!(t.getCause() instanceof JobCancellationException)) { // we'll cancel the job
-						error.f0 = t;
-					}
-				}
-			}
-		};
-		Thread jobThread = new Thread(job);
-		jobThread.start();
-
-		try {
-			// connect to JMX
-			MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer();
-			// wait until we've found all 5 offset metrics
-			Set<ObjectName> offsetMetrics = mBeanServer.queryNames(new ObjectName("*current-offsets*:*"), null);
-			while (offsetMetrics.size() < 5) { // test will time out if metrics are not properly working
-				if (error.f0 != null) {
-					// fail test early
-					throw error.f0;
-				}
-				offsetMetrics = mBeanServer.queryNames(new ObjectName("*current-offsets*:*"), null);
-				Thread.sleep(50);
-			}
-			Assert.assertEquals(5, offsetMetrics.size());
-			// we can't rely on the consumer to have touched all the partitions already
-			// that's why we'll wait until all five partitions have a positive offset.
-			// The test will fail if we never meet the condition
-			while (true) {
-				int numPosOffsets = 0;
-				// check that offsets are correctly reported
-				for (ObjectName object : offsetMetrics) {
-					Object offset = mBeanServer.getAttribute(object, "Value");
-					if((long) offset >= 0) {
-						numPosOffsets++;
-					}
-				}
-				if (numPosOffsets == 5) {
-					break;
-				}
-				// wait for the consumer to consume on all partitions
-				Thread.sleep(50);
-			}
-
-			// check if producer metrics are also available.
-			Set<ObjectName> producerMetrics = mBeanServer.queryNames(new ObjectName("*KafkaProducer*:*"), null);
-			Assert.assertTrue("No producer metrics found", producerMetrics.size() > 30);
-
-
-			LOG.info("Found all JMX metrics. Cancelling job.");
-		} finally {
-			// cancel
-			JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
-		}
-
-		while (jobThread.isAlive()) {
-			Thread.sleep(50);
-		}
-		if (error.f0 != null) {
-			throw error.f0;
-		}
-
-		deleteTestTopic(topic);
-	}
-
-
-	public static class FixedNumberDeserializationSchema implements DeserializationSchema<Tuple2<Integer, Integer>> {
-		
-		final int finalCount;
-		int count = 0;
-		
-		TypeInformation<Tuple2<Integer, Integer>> ti = TypeInfoParser.parse("Tuple2<Integer, Integer>");
-		TypeSerializer<Tuple2<Integer, Integer>> ser = ti.createSerializer(new ExecutionConfig());
-
-		public FixedNumberDeserializationSchema(int finalCount) {
-			this.finalCount = finalCount;
-		}
-
-		@Override
-		public Tuple2<Integer, Integer> deserialize(byte[] message) throws IOException {
-			DataInputView in = new DataInputViewStreamWrapper(new ByteArrayInputStream(message));
-			return ser.deserialize(in);
-		}
-
-		@Override
-		public boolean isEndOfStream(Tuple2<Integer, Integer> nextElement) {
-			return ++count >= finalCount;
-		}
-
-		@Override
-		public TypeInformation<Tuple2<Integer, Integer>> getProducedType() {
-			return ti;
-		}
-	}
-
-
-	// ------------------------------------------------------------------------
-	//  Reading writing test data sets
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Runs a job using the provided environment to read a sequence of records from a single Kafka topic.
-	 * The method allows to individually specify the expected starting offset and total read value count of each partition.
-	 * The job will be considered successful only if all partition read results match the start offset and value count criteria.
-	 */
-	protected void readSequence(StreamExecutionEnvironment env, Properties cc,
-								final String topicName,
-								final Map<Integer, Tuple2<Integer, Integer>> partitionsToValuesCountAndStartOffset) throws Exception {
-		final int sourceParallelism = partitionsToValuesCountAndStartOffset.keySet().size();
-
-		int finalCountTmp = 0;
-		for (Map.Entry<Integer, Tuple2<Integer, Integer>> valuesCountAndStartOffset : partitionsToValuesCountAndStartOffset.entrySet()) {
-			finalCountTmp += valuesCountAndStartOffset.getValue().f0;
-		}
-		final int finalCount = finalCountTmp;
-
-		final TypeInformation<Tuple2<Integer, Integer>> intIntTupleType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
-
-		final TypeInformationSerializationSchema<Tuple2<Integer, Integer>> deser =
-			new TypeInformationSerializationSchema<>(intIntTupleType, env.getConfig());
-
-		// create the consumer
-		cc.putAll(secureProps);
-		FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> consumer = kafkaServer.getConsumer(topicName, deser, cc);
-
-		DataStream<Tuple2<Integer, Integer>> source = env
-			.addSource(consumer).setParallelism(sourceParallelism)
-			.map(new ThrottledMapper<Tuple2<Integer, Integer>>(20)).setParallelism(sourceParallelism);
-
-		// verify data
-		source.flatMap(new RichFlatMapFunction<Tuple2<Integer, Integer>, Integer>() {
-
-			private HashMap<Integer, BitSet> partitionsToValueCheck;
-			private int count = 0;
-
-			@Override
-			public void open(Configuration parameters) throws Exception {
-				partitionsToValueCheck = new HashMap<>();
-				for (Integer partition : partitionsToValuesCountAndStartOffset.keySet()) {
-					partitionsToValueCheck.put(partition, new BitSet());
-				}
-			}
-
-			@Override
-			public void flatMap(Tuple2<Integer, Integer> value, Collector<Integer> out) throws Exception {
-				int partition = value.f0;
-				int val = value.f1;
-
-				BitSet bitSet = partitionsToValueCheck.get(partition);
-				if (bitSet == null) {
-					throw new RuntimeException("Got a record from an unknown partition");
-				} else {
-					bitSet.set(val - partitionsToValuesCountAndStartOffset.get(partition).f1);
-				}
-
-				count++;
-
-				LOG.info("Received message {}, total {} messages", value, count);
-
-				// verify if we've seen everything
-				if (count == finalCount) {
-					for (Map.Entry<Integer, BitSet> partitionsToValueCheck : this.partitionsToValueCheck.entrySet()) {
-						BitSet check = partitionsToValueCheck.getValue();
-						int expectedValueCount = partitionsToValuesCountAndStartOffset.get(partitionsToValueCheck.getKey()).f0;
-
-						if (check.cardinality() != expectedValueCount) {
-							throw new RuntimeException("Expected cardinality to be " + expectedValueCount +
-								", but was " + check.cardinality());
-						} else if (check.nextClearBit(0) != expectedValueCount) {
-							throw new RuntimeException("Expected next clear bit to be " + expectedValueCount +
-								", but was " + check.cardinality());
-						}
-					}
-
-					// test has passed
-					throw new SuccessException();
-				}
-			}
-
-		}).setParallelism(1);
-
-		tryExecute(env, "Read data from Kafka");
-
-		LOG.info("Successfully read sequence for verification");
-	}
-
-	/**
-	 * Variant of {@link KafkaConsumerTestBase#readSequence(StreamExecutionEnvironment, Properties, String, Map)} to
-	 * expect reading from the same start offset and the same value count for all partitions of a single Kafka topic.
-	 */
-	protected void readSequence(StreamExecutionEnvironment env, Properties cc,
-								final int sourceParallelism,
-								final String topicName,
-								final int valuesCount, final int startFrom) throws Exception {
-		HashMap<Integer, Tuple2<Integer, Integer>> partitionsToValuesCountAndStartOffset = new HashMap<>();
-		for (int i = 0; i < sourceParallelism; i++) {
-			partitionsToValuesCountAndStartOffset.put(i, new Tuple2<>(valuesCount, startFrom));
-		}
-		readSequence(env, cc, topicName, partitionsToValuesCountAndStartOffset);
-	}
-
-	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" +
-				"===================================");
-
-		final TypeInformation<Tuple2<Integer, Integer>> resultType = 
-				TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {});
-
-		final KeyedSerializationSchema<Tuple2<Integer, Integer>> serSchema =
-				new KeyedSerializationSchemaWrapper<>(
-						new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig()));
-
-		final KeyedDeserializationSchema<Tuple2<Integer, Integer>> deserSchema =
-				new KeyedDeserializationSchemaWrapper<>(
-						new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig()));
-		
-		final int maxNumAttempts = 10;
-
-		for (int attempt = 1; attempt <= maxNumAttempts; attempt++) {
-			
-			final String topicName = baseTopicName + '-' + attempt;
-			
-			LOG.info("Writing attempt #1");
-			
-			// -------- Write the Sequence --------
-
-			createTestTopic(topicName, parallelism, replicationFactor);
-
-			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");
-			producerProperties.putAll(secureProps);
-			
-			kafkaServer.produceIntoKafka(stream, topicName, serSchema, producerProperties, new Tuple2Partitioner(parallelism))
-					.setParallelism(parallelism);
-
-			try {
-				writeEnv.execute("Write sequence");
-			}
-			catch (Exception e) {
-				LOG.error("Write attempt failed, trying again", e);
-				deleteTestTopic(topicName);
-				JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout));
-				continue;
-			}
-			
-			LOG.info("Finished writing sequence");
-
-			// -------- Validate the Sequence --------
-			
-			// we need to validate the sequence, because kafka's producers are not exactly once
-			LOG.info("Validating sequence");
-
-			JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout));
-			
-			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");
-			readProps.putAll(secureProps);
-			FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> consumer = kafkaServer.getConsumer(topicName, deserSchema, readProps);
-
-			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;
-				}
-			}
-
-			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");
-	}
-
-	// ------------------------------------------------------------------------
-	//  Debugging utilities
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Read topic to list, only using Kafka code.
-	 */
-	private static List<MessageAndMetadata<byte[], byte[]>> readTopicToList(String topicName, ConsumerConfig config, final int stopAfter) {
-		ConsumerConnector consumerConnector = Consumer.createJavaConsumerConnector(config);
-		// we request only one stream per consumer instance. Kafka will make sure that each consumer group
-		// will see each message only once.
-		Map<String,Integer> topicCountMap = Collections.singletonMap(topicName, 1);
-		Map<String, List<KafkaStream<byte[], byte[]>>> streams = consumerConnector.createMessageStreams(topicCountMap);
-		if (streams.size() != 1) {
-			throw new RuntimeException("Expected only one message stream but got "+streams.size());
-		}
-		List<KafkaStream<byte[], byte[]>> kafkaStreams = streams.get(topicName);
-		if (kafkaStreams == null) {
-			throw new RuntimeException("Requested stream not available. Available streams: "+streams.toString());
-		}
-		if (kafkaStreams.size() != 1) {
-			throw new RuntimeException("Requested 1 stream from Kafka, bot got "+kafkaStreams.size()+" streams");
-		}
-		LOG.info("Opening Consumer instance for topic '{}' on group '{}'", topicName, config.groupId());
-		ConsumerIterator<byte[], byte[]> iteratorToRead = kafkaStreams.get(0).iterator();
-
-		List<MessageAndMetadata<byte[], byte[]>> result = new ArrayList<>();
-		int read = 0;
-		while(iteratorToRead.hasNext()) {
-			read++;
-			result.add(iteratorToRead.next());
-			if (read == stopAfter) {
-				LOG.info("Read "+read+" elements");
-				return result;
-			}
-		}
-		return result;
-	}
-
-	private static void printTopic(String topicName, ConsumerConfig config,
-								DeserializationSchema<?> deserializationSchema,
-								int stopAfter) throws IOException {
-
-		List<MessageAndMetadata<byte[], byte[]>> contents = readTopicToList(topicName, config, stopAfter);
-		LOG.info("Printing contents of topic {} in consumer grouo {}", topicName, config.groupId());
-
-		for (MessageAndMetadata<byte[], byte[]> message: contents) {
-			Object out = deserializationSchema.deserialize(message.message());
-			LOG.info("Message: partition: {} offset: {} msg: {}", message.partition(), message.offset(), out.toString());
-		}
-	}
-
-	private static void printTopic(String topicName, int elements,DeserializationSchema<?> deserializer) 
-			throws IOException
-	{
-		// write the sequence to log for debugging purposes
-		Properties newProps = new Properties(standardProps);
-		newProps.setProperty("group.id", "topic-printer"+ UUID.randomUUID().toString());
-		newProps.setProperty("auto.offset.reset", "smallest");
-		newProps.setProperty("zookeeper.connect", standardProps.getProperty("zookeeper.connect"));
-		newProps.putAll(secureProps);
-
-		ConsumerConfig printerConfig = new ConsumerConfig(newProps);
-		printTopic(topicName, printerConfig, deserializer, elements);
-	}
-
-
-	public static class BrokerKillingMapper<T> extends RichMapFunction<T,T>
-			implements Checkpointed<Integer>, CheckpointListener {
-
-		private static final long serialVersionUID = 6334389850158707313L;
-
-		public static volatile boolean killedLeaderBefore;
-		public static volatile boolean hasBeenCheckpointedBeforeFailure;
-		
-		private final int shutdownBrokerId;
-		private final int failCount;
-		private int numElementsTotal;
-
-		private boolean failer;
-		private boolean hasBeenCheckpointed;
-
-
-		public BrokerKillingMapper(int shutdownBrokerId, int failCount) {
-			this.shutdownBrokerId = shutdownBrokerId;
-			this.failCount = failCount;
-		}
-
-		@Override
-		public void open(Configuration parameters) {
-			failer = getRuntimeContext().getIndexOfThisSubtask() == 0;
-		}
-
-		@Override
-		public T map(T value) throws Exception {
-			numElementsTotal++;
-			
-			if (!killedLeaderBefore) {
-				Thread.sleep(10);
-				
-				if (failer && numElementsTotal >= failCount) {
-					// shut down a Kafka broker
-					KafkaServer toShutDown = null;
-					for (KafkaServer server : kafkaServer.getBrokers()) {
-
-						if (kafkaServer.getBrokerId(server) == shutdownBrokerId) {
-							toShutDown = server;
-							break;
-						}
-					}
-	
-					if (toShutDown == null) {
-						StringBuilder listOfBrokers = new StringBuilder();
-						for (KafkaServer server : kafkaServer.getBrokers()) {
-							listOfBrokers.append(kafkaServer.getBrokerId(server));
-							listOfBrokers.append(" ; ");
-						}
-						
-						throw new Exception("Cannot find broker to shut down: " + shutdownBrokerId
-								+ " ; available brokers: " + listOfBrokers.toString());
-					}
-					else {
-						hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed;
-						killedLeaderBefore = true;
-						toShutDown.shutdown();
-					}
-				}
-			}
-			return value;
-		}
-
-		@Override
-		public void notifyCheckpointComplete(long checkpointId) {
-			hasBeenCheckpointed = true;
-		}
-
-		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return numElementsTotal;
-		}
-
-		@Override
-		public void restoreState(Integer state) {
-			this.numElementsTotal = state;
-		}
-	}
-}

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


[07/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index cf39606..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
+++ /dev/null
@@ -1,552 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.metrics.Gauge;
-import org.apache.flink.metrics.MetricGroup;
-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.watermark.Watermark;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-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> {
-	
-	protected static final int NO_TIMESTAMPS_WATERMARKS = 0;
-	protected static final int PERIODIC_WATERMARKS = 1;
-	protected static final int PUNCTUATED_WATERMARKS = 2;
-	
-	// ------------------------------------------------------------------------
-	
-	/** The source context to emit records and watermarks to */
-	protected final SourceContext<T> sourceContext;
-
-	/** The lock that guarantees that record emission and state updates are atomic,
-	 * from the view of taking a checkpoint */
-	protected 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 */
-	protected final int timestampWatermarkMode;
-
-	/** Flag whether to register metrics for the fetcher */
-	protected final boolean useMetrics;
-
-	/** 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,
-			ProcessingTimeService processingTimeProvider,
-			long autoWatermarkInterval,
-			ClassLoader userCodeClassLoader,
-			boolean useMetrics) throws Exception
-	{
-		this.sourceContext = checkNotNull(sourceContext);
-		this.checkpointLock = sourceContext.getCheckpointLock();
-		this.useMetrics = useMetrics;
-		
-		// 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,
-				userCodeClassLoader);
-		
-		// if we have periodic watermarks, kick off the interval scheduler
-		if (timestampWatermarkMode == PERIODIC_WATERMARKS) {
-			KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[] parts = 
-					(KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[]) allPartitions;
-			
-			PeriodicWatermarkEmitter periodicEmitter = 
-					new PeriodicWatermarkEmitter(parts, sourceContext, processingTimeProvider, autoWatermarkInterval);
-			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). The given offsets are the internal checkpointed offsets, representing
-	 * the last processed record of each partition. Version-specific implementations of this method
-	 * need to hold the contract that the given offsets must be incremented by 1 before
-	 * committing them, so that committed offsets to Kafka represent "the next record to process".
-	 * 
-	 * @param offsets The offsets to commit to Kafka (implementations must increment offsets by 1 before committing).
-	 * @throws Exception This method forwards exceptions.
-	 */
-	public abstract void commitInternalOffsetsToKafka(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()) {
-			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
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Emits a record without attaching an existing timestamp to it.
-	 * 
-	 * <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 of the record
-	 */
-	protected void emitRecord(T record, KafkaTopicPartitionState<KPH> partitionState, long offset) throws Exception {
-		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, Long.MIN_VALUE);
-		}
-		else {
-			emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, Long.MIN_VALUE);
-		}
-	}
-
-	/**
-	 * Emits a record attaching a timestamp to it.
-	 *
-	 * <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 of the record
-	 */
-	protected void emitRecordWithTimestamp(
-			T record, KafkaTopicPartitionState<KPH> partitionState, long offset, long timestamp) throws Exception {
-
-		if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS) {
-			// fast path logic, in case there are no watermarks generated in the fetcher
-
-			// emit the record, using the checkpoint lock to guarantee
-			// atomicity of record emission and offset state update
-			synchronized (checkpointLock) {
-				sourceContext.collectWithTimestamp(record, timestamp);
-				partitionState.setOffset(offset);
-			}
-		}
-		else if (timestampWatermarkMode == PERIODIC_WATERMARKS) {
-			emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset, timestamp);
-		}
-		else {
-			emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, timestamp);
-		}
-	}
-
-	/**
-	 * Record emission, if a timestamp will be attached from an assigner that is
-	 * also a periodic watermark generator.
-	 */
-	protected void emitRecordWithTimestampAndPeriodicWatermark(
-			T record, KafkaTopicPartitionState<KPH> partitionState, long offset, long kafkaEventTimestamp)
-	{
-		@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, kafkaEventTimestamp);
-		}
-
-		// 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.
-	 */
-	protected void emitRecordWithTimestampAndPunctuatedWatermark(
-			T record, KafkaTopicPartitionState<KPH> partitionState, long offset, long kafkaEventTimestamp)
-	{
-		@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, kafkaEventTimestamp);
-		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
-	{
-		switch (timestampWatermarkMode) {
-			
-			case NO_TIMESTAMPS_WATERMARKS: {
-				@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);
-					partitions[pos++] = new KafkaTopicPartitionState<>(partition, kafkaHandle);
-				}
-
-				return partitions;
-			}
-
-			case PERIODIC_WATERMARKS: {
-				@SuppressWarnings("unchecked")
-				KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH>[] partitions =
-						(KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH>[])
-								new KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[assignedPartitions.size()];
-
-				int pos = 0;
-				for (KafkaTopicPartition partition : assignedPartitions) {
-					KPH kafkaHandle = createKafkaPartitionHandle(partition);
-
-					AssignerWithPeriodicWatermarks<T> assignerInstance =
-							watermarksPeriodic.deserializeValue(userCodeClassLoader);
-					
-					partitions[pos++] = new KafkaTopicPartitionStateWithPeriodicWatermarks<>(
-							partition, kafkaHandle, assignerInstance);
-				}
-
-				return partitions;
-			}
-
-			case PUNCTUATED_WATERMARKS: {
-				@SuppressWarnings("unchecked")
-				KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH>[] partitions =
-						(KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH>[])
-								new KafkaTopicPartitionStateWithPunctuatedWatermarks<?, ?>[assignedPartitions.size()];
-
-				int pos = 0;
-				for (KafkaTopicPartition partition : assignedPartitions) {
-					KPH kafkaHandle = createKafkaPartitionHandle(partition);
-
-					AssignerWithPunctuatedWatermarks<T> assignerInstance =
-							watermarksPunctuated.deserializeValue(userCodeClassLoader);
-
-					partitions[pos++] = new KafkaTopicPartitionStateWithPunctuatedWatermarks<>(
-							partition, kafkaHandle, assignerInstance);
-				}
-
-				return partitions;
-			}
-			default:
-				// cannot happen, add this as a guard for the future
-				throw new RuntimeException();
-		}
-	}
-
-	// ------------------------- Metrics ----------------------------------
-
-	/**
-	 * Add current and committed offsets to metric group
-	 *
-	 * @param metricGroup The metric group to use
-	 */
-	protected void addOffsetStateGauge(MetricGroup metricGroup) {
-		// add current offsets to gage
-		MetricGroup currentOffsets = metricGroup.addGroup("current-offsets");
-		MetricGroup committedOffsets = metricGroup.addGroup("committed-offsets");
-		for (KafkaTopicPartitionState<?> ktp: subscribedPartitions()) {
-			currentOffsets.gauge(ktp.getTopic() + "-" + ktp.getPartition(), new OffsetGauge(ktp, OffsetGaugeType.CURRENT_OFFSET));
-			committedOffsets.gauge(ktp.getTopic() + "-" + ktp.getPartition(), new OffsetGauge(ktp, OffsetGaugeType.COMMITTED_OFFSET));
-		}
-	}
-
-	/**
-	 * Gauge types
-	 */
-	private enum OffsetGaugeType {
-		CURRENT_OFFSET,
-		COMMITTED_OFFSET
-	}
-
-	/**
-	 * Gauge for getting the offset of a KafkaTopicPartitionState.
-	 */
-	private static class OffsetGauge implements Gauge<Long> {
-
-		private final KafkaTopicPartitionState<?> ktp;
-		private final OffsetGaugeType gaugeType;
-
-		public OffsetGauge(KafkaTopicPartitionState<?> ktp, OffsetGaugeType gaugeType) {
-			this.ktp = ktp;
-			this.gaugeType = gaugeType;
-		}
-
-		@Override
-		public Long getValue() {
-			switch(gaugeType) {
-				case COMMITTED_OFFSET:
-					return ktp.getCommittedOffset();
-				case CURRENT_OFFSET:
-					return ktp.getOffset();
-				default:
-					throw new RuntimeException("Unknown gauge type: " + gaugeType);
-			}
-		}
-	}
- 	// ------------------------------------------------------------------------
-	
-	/**
-	 * 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 ProcessingTimeCallback {
-
-		private final KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[] allPartitions;
-		
-		private final SourceContext<?> emitter;
-		
-		private final ProcessingTimeService timerService;
-
-		private final long interval;
-		
-		private long lastWatermarkTimestamp;
-		
-		//-------------------------------------------------
-
-		PeriodicWatermarkEmitter(
-				KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[] allPartitions,
-				SourceContext<?> emitter,
-				ProcessingTimeService timerService,
-				long autoWatermarkInterval)
-		{
-			this.allPartitions = checkNotNull(allPartitions);
-			this.emitter = checkNotNull(emitter);
-			this.timerService = checkNotNull(timerService);
-			this.interval = autoWatermarkInterval;
-			this.lastWatermarkTimestamp = Long.MIN_VALUE;
-		}
-
-		//-------------------------------------------------
-		
-		public void start() {
-			timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
-		}
-		
-		@Override
-		public void onProcessingTime(long timestamp) throws Exception {
-
-			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
-			timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index c736493..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import javax.annotation.Nullable;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * A proxy that communicates exceptions between threads. Typically used if an exception
- * from a spawned thread needs to be recognized by the "parent" (spawner) thread.
- * 
- * <p>The spawned thread would set the exception via {@link #reportError(Throwable)}.
- * The parent would check (at certain points) for exceptions via {@link #checkAndThrowException()}.
- * Optionally, the parent can pass itself in the constructor to be interrupted as soon as
- * an exception occurs.
- * 
- * <pre>
- * {@code
- * 
- * final ExceptionProxy errorProxy = new ExceptionProxy(Thread.currentThread());
- * 
- * Thread subThread = new Thread() {
- * 
- *     public void run() {
- *         try {
- *             doSomething();
- *         } catch (Throwable t) {
- *             errorProxy.reportError(
- *         } finally {
- *             doSomeCleanup();
- *         }
- *     }
- * };
- * subThread.start();
- * 
- * doSomethingElse();
- * errorProxy.checkAndThrowException();
- * 
- * doSomethingMore();
- * errorProxy.checkAndThrowException();
- * 
- * try {
- *     subThread.join();
- * } catch (InterruptedException e) {
- *     errorProxy.checkAndThrowException();
- *     // restore interrupted status, if not caused by an exception
- *     Thread.currentThread().interrupt();
- * }
- * }
- * </pre>
- */
-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;
-
-	/**
-	 * Creates an exception proxy that interrupts the given thread upon
-	 * report of an exception. The thread to interrupt may be null.
-	 * 
-	 * @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 and interrupts the target thread,
-	 * if no other exception has occurred so far.
-	 * 
-	 * <p>The exception is only set (and the interruption is only triggered),
-	 * if no other exception was set before.
-	 * 
-	 * @param t The exception that occurred
-	 */
-	public void reportError(Throwable t) {
-		// set the exception, if it is the first (and the exception is non null)
-		if (t != null && exception.compareAndSet(null, t) && toInterrupt != null) {
-			toInterrupt.interrupt();
-		}
-	}
-
-	/**
-	 * Checks whether an exception has been set via {@link #reportError(Throwable)}.
-	 * If yes, that exception if re-thrown by this method.
-	 * 
-	 * @throws Exception This method re-throws the exception, if set.
-	 */
-	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/de4fe3b7/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
deleted file mode 100644
index c68fe28..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * 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 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;
-	private final int cachedHash;
-
-	public KafkaTopicPartition(String topic, int partition) {
-		this.topic = requireNonNull(topic);
-		this.partition = partition;
-		this.cachedHash = 31 * topic.hashCode() + partition;
-	}
-
-	// ------------------------------------------------------------------------
-	
-	public String getTopic() {
-		return topic;
-	}
-
-	public int getPartition() {
-		return partition;
-	}
-
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public String toString() {
-		return "KafkaTopicPartition{" +
-				"topic='" + topic + '\'' +
-				", partition=" + partition +
-				'}';
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		else if (o instanceof KafkaTopicPartition) {
-			KafkaTopicPartition that = (KafkaTopicPartition) o;
-			return this.partition == that.partition && this.topic.equals(that.topic);
-		}
-		else {
-			return false;
-		}
-	}
-
-	@Override
-	public int hashCode() {
-		return cachedHash;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	public static String toString(Map<KafkaTopicPartition, Long> map) {
-		StringBuilder sb = new StringBuilder();
-		for (Map.Entry<KafkaTopicPartition, Long> p: map.entrySet()) {
-			KafkaTopicPartition ktp = p.getKey();
-			sb.append(ktp.getTopic()).append(":").append(ktp.getPartition()).append("=").append(p.getValue()).append(", ");
-		}
-		return sb.toString();
-	}
-
-	public static String toString(List<KafkaTopicPartition> partitions) {
-		StringBuilder sb = new StringBuilder();
-		for (KafkaTopicPartition p: partitions) {
-			sb.append(p.getTopic()).append(":").append(p.getPartition()).append(", ");
-		}
-		return sb.toString();
-	}
-
-
-	public static List<KafkaTopicPartition> dropLeaderData(List<KafkaTopicPartitionLeader> partitionInfos) {
-		List<KafkaTopicPartition> ret = new ArrayList<>(partitionInfos.size());
-		for(KafkaTopicPartitionLeader ktpl: partitionInfos) {
-			ret.add(ktpl.getTopicPartition());
-		}
-		return ret;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
deleted file mode 100644
index 1959a05..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import org.apache.kafka.common.Node;
-
-import java.io.Serializable;
-
-/**
- * Serializable Topic Partition info with leader Node information.
- * This class is used at runtime.
- */
-public class KafkaTopicPartitionLeader implements Serializable {
-
-	private static final long serialVersionUID = 9145855900303748582L;
-
-	private final int leaderId;
-	private final int leaderPort;
-	private final String leaderHost;
-	private final KafkaTopicPartition topicPartition;
-	private final int cachedHash;
-
-	public KafkaTopicPartitionLeader(KafkaTopicPartition topicPartition, Node leader) {
-		this.topicPartition = topicPartition;
-		if (leader == null) {
-			this.leaderId = -1;
-			this.leaderHost = null;
-			this.leaderPort = -1;
-		} else {
-			this.leaderId = leader.id();
-			this.leaderPort = leader.port();
-			this.leaderHost = leader.host();
-		}
-		int cachedHash = (leader == null) ? 14 : leader.hashCode();
-		this.cachedHash = 31 * cachedHash + topicPartition.hashCode();
-	}
-
-	public KafkaTopicPartition getTopicPartition() {
-		return topicPartition;
-	}
-
-	public Node getLeader() {
-		if (this.leaderId == -1) {
-			return null;
-		} else {
-			return new Node(leaderId, leaderHost, leaderPort);
-		}
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (!(o instanceof KafkaTopicPartitionLeader)) {
-			return false;
-		}
-
-		KafkaTopicPartitionLeader that = (KafkaTopicPartitionLeader) o;
-
-		if (!topicPartition.equals(that.topicPartition)) {
-			return false;
-		}
-		return leaderId == that.leaderId && leaderPort == that.leaderPort && leaderHost.equals(that.leaderHost);
-	}
-
-	@Override
-	public int hashCode() {
-		return cachedHash;
-	}
-
-	@Override
-	public String toString() {
-		return "KafkaTopicPartitionLeader{" +
-				"leaderId=" + leaderId +
-				", leaderPort=" + leaderPort +
-				", leaderHost='" + leaderHost + '\'' +
-				", topic=" + topicPartition.getTopic() +
-				", partition=" + topicPartition.getPartition() +
-				'}';
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index 7cb5f46..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.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;
-
-	/** The offset of the Kafka partition that has been committed */
-	private volatile long committedOffset;
-
-	// ------------------------------------------------------------------------
-	
-	public KafkaTopicPartitionState(KafkaTopicPartition partition, KPH kafkaPartitionHandle) {
-		this.partition = partition;
-		this.kafkaPartitionHandle = kafkaPartitionHandle;
-		this.offset = OFFSET_NOT_SET;
-		this.committedOffset = 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;
-	}
-
-	public final void setCommittedOffset(long offset) {
-		this.committedOffset = offset;
-	}
-
-	public final long getCommittedOffset() {
-		return committedOffset;
-	}
-
-	
-	// ------------------------------------------------------------------------
-
-	@Override
-	public String toString() {
-		return "Partition: " + partition + ", KafkaPartitionHandle=" + kafkaPartitionHandle
-				+ ", offset=" + (isOffsetDefined() ? String.valueOf(offset) : "(not set)");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index efdc73f..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.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, long kafkaEventTimestamp) {
-		return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp);
-	}
-	
-	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/de4fe3b7/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
deleted file mode 100644
index edf40ce..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.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, long kafkaEventTimestamp) {
-		return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp);
-	}
-
-	@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/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TypeUtil.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TypeUtil.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TypeUtil.java
deleted file mode 100644
index 7a41ade..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TypeUtil.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-
-public class TypeUtil {
-	private TypeUtil() {}
-
-	/**
-	 * Creates TypeInformation array for an array of Classes.
-	 * @param fieldTypes classes to extract type information from
-	 * @return type information
-	 */
-	public static TypeInformation<?>[] toTypeInfo(Class<?>[] fieldTypes) {
-		TypeInformation<?>[] typeInfos = new TypeInformation[fieldTypes.length];
-		for (int i = 0; i < fieldTypes.length; i++) {
-			typeInfos[i] = TypeExtractor.getForClass(fieldTypes[i]);
-		}
-		return typeInfos;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java
deleted file mode 100644
index cedb696..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals.metrics;
-
-import org.apache.flink.metrics.Gauge;
-
-/**
- * Gauge for getting the current value of a Kafka metric.
- */
-public class KafkaMetricWrapper implements Gauge<Double> {
-	private final org.apache.kafka.common.Metric kafkaMetric;
-
-	public KafkaMetricWrapper(org.apache.kafka.common.Metric metric) {
-		this.kafkaMetric = metric;
-	}
-
-	@Override
-	public Double getValue() {
-		return kafkaMetric.value();
-	}
-}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
deleted file mode 100644
index d170058..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.util.serialization;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-import java.io.IOException;
-
-
-/**
- * DeserializationSchema that deserializes a JSON String into an ObjectNode.
- * <p>
- * Fields can be accessed by calling objectNode.get(&lt;name>).as(&lt;type>)
- */
-public class JSONDeserializationSchema extends AbstractDeserializationSchema<ObjectNode> {
-	private ObjectMapper mapper;
-
-	@Override
-	public ObjectNode deserialize(byte[] message) throws IOException {
-		if (mapper == null) {
-			mapper = new ObjectMapper();
-		}
-		return mapper.readValue(message, ObjectNode.class);
-	}
-
-	@Override
-	public boolean isEndOfStream(ObjectNode nextElement) {
-		return false;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java
deleted file mode 100644
index 261a111..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.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.util.serialization;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-import java.io.IOException;
-
-import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass;
-
-/**
- * DeserializationSchema that deserializes a JSON String into an ObjectNode.
- * <p>
- * Key fields can be accessed by calling objectNode.get("key").get(&lt;name>).as(&lt;type>)
- * <p>
- * Value fields can be accessed by calling objectNode.get("value").get(&lt;name>).as(&lt;type>)
- * <p>
- * Metadata fields can be accessed by calling objectNode.get("metadata").get(&lt;name>).as(&lt;type>) and include
- * the "offset" (long), "topic" (String) and "partition" (int).
- */
-public class JSONKeyValueDeserializationSchema implements KeyedDeserializationSchema<ObjectNode> {
-	private final boolean includeMetadata;
-	private ObjectMapper mapper;
-
-	public JSONKeyValueDeserializationSchema(boolean includeMetadata) {
-		this.includeMetadata = includeMetadata;
-	}
-
-	@Override
-	public ObjectNode deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
-		if (mapper == null) {
-			mapper = new ObjectMapper();
-		}
-		ObjectNode node = mapper.createObjectNode();
-		node.set("key", mapper.readValue(messageKey, JsonNode.class));
-		node.set("value", mapper.readValue(message, JsonNode.class));
-		if (includeMetadata) {
-			node.putObject("metadata")
-				.put("offset", offset)
-				.put("topic", topic)
-				.put("partition", partition);
-		}
-		return node;
-	}
-
-	@Override
-	public boolean isEndOfStream(ObjectNode nextElement) {
-		return false;
-	}
-
-	@Override
-	public TypeInformation<ObjectNode> getProducedType() {
-		return getForClass(ObjectNode.class);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
deleted file mode 100644
index 4344810..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.typeutils.RowTypeInfo;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-
-/**
- * Deserialization schema from JSON to {@link Row}.
- *
- * <p>Deserializes the <code>byte[]</code> messages as a JSON object and reads
- * the specified fields.
- *
- * <p>Failure during deserialization are forwarded as wrapped IOExceptions.
- */
-public class JsonRowDeserializationSchema implements DeserializationSchema<Row> {
-
-	/** Field names to parse. Indices match fieldTypes indices. */
-	private final String[] fieldNames;
-
-	/** Types to parse fields as. Indices match fieldNames indices. */
-	private final TypeInformation<?>[] fieldTypes;
-
-	/** Object mapper for parsing the JSON. */
-	private final ObjectMapper objectMapper = new ObjectMapper();
-
-	/** Flag indicating whether to fail on a missing field. */
-	private boolean failOnMissingField;
-
-	/**
-	 * Creates a JSON deserialization schema for the given fields and type classes.
-	 *
-	 * @param fieldNames Names of JSON fields to parse.
-	 * @param fieldTypes Type classes to parse JSON fields as.
-	 */
-	public JsonRowDeserializationSchema(String[] fieldNames, Class<?>[] fieldTypes) {
-		this.fieldNames = Preconditions.checkNotNull(fieldNames, "Field names");
-
-		this.fieldTypes = new TypeInformation[fieldTypes.length];
-		for (int i = 0; i < fieldTypes.length; i++) {
-			this.fieldTypes[i] = TypeExtractor.getForClass(fieldTypes[i]);
-		}
-
-		Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
-				"Number of provided field names and types does not match.");
-	}
-
-	/**
-	 * Creates a JSON deserialization schema for the given fields and types.
-	 *
-	 * @param fieldNames Names of JSON fields to parse.
-	 * @param fieldTypes Types to parse JSON fields as.
-	 */
-	public JsonRowDeserializationSchema(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
-		this.fieldNames = Preconditions.checkNotNull(fieldNames, "Field names");
-		this.fieldTypes = Preconditions.checkNotNull(fieldTypes, "Field types");
-
-		Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
-				"Number of provided field names and types does not match.");
-	}
-
-	@Override
-	public Row deserialize(byte[] message) throws IOException {
-		try {
-			JsonNode root = objectMapper.readTree(message);
-
-			Row row = new Row(fieldNames.length);
-			for (int i = 0; i < fieldNames.length; i++) {
-				JsonNode node = root.get(fieldNames[i]);
-
-				if (node == null) {
-					if (failOnMissingField) {
-						throw new IllegalStateException("Failed to find field with name '"
-								+ fieldNames[i] + "'.");
-					} else {
-						row.setField(i, null);
-					}
-				} else {
-					// Read the value as specified type
-					Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass());
-					row.setField(i, value);
-				}
-			}
-
-			return row;
-		} catch (Throwable t) {
-			throw new IOException("Failed to deserialize JSON object.", t);
-		}
-	}
-
-	@Override
-	public boolean isEndOfStream(Row nextElement) {
-		return false;
-	}
-
-	@Override
-	public TypeInformation<Row> getProducedType() {
-		return new RowTypeInfo(fieldTypes);
-	}
-
-	/**
-	 * Configures the failure behaviour if a JSON field is missing.
-	 *
-	 * <p>By default, a missing field is ignored and the field is set to null.
-	 *
-	 * @param failOnMissingField Flag indicating whether to fail or not on a missing field.
-	 */
-	public void setFailOnMissingField(boolean failOnMissingField) {
-		this.failOnMissingField = failOnMissingField;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
deleted file mode 100644
index 077ff13..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.util.serialization;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.util.Preconditions;
-
-
-/**
- * Serialization schema that serializes an object into a JSON bytes.
- *
- * <p>Serializes the input {@link Row} object into a JSON string and
- * converts it into <code>byte[]</code>.
- *
- * <p>Result <code>byte[]</code> messages can be deserialized using
- * {@link JsonRowDeserializationSchema}.
- */
-public class JsonRowSerializationSchema implements SerializationSchema<Row> {
-	/** Fields names in the input Row object */
-	private final String[] fieldNames;
-	/** Object mapper that is used to create output JSON objects */
-	private static ObjectMapper mapper = new ObjectMapper();
-
-	/**
-	 * Creates a JSON serialization schema for the given fields and types.
-	 *
-	 * @param fieldNames Names of JSON fields to parse.
-	 */
-	public JsonRowSerializationSchema(String[] fieldNames) {
-		this.fieldNames = Preconditions.checkNotNull(fieldNames);
-	}
-
-	@Override
-	public byte[] serialize(Row row) {
-		if (row.productArity() != fieldNames.length) {
-			throw new IllegalStateException(String.format(
-				"Number of elements in the row %s is different from number of field names: %d", row, fieldNames.length));
-		}
-
-		ObjectNode objectNode = mapper.createObjectNode();
-
-		for (int i = 0; i < row.productArity(); i++) {
-			JsonNode node = mapper.valueToTree(row.productElement(i));
-			objectNode.set(fieldNames[i], node);
-		}
-
-		try {
-			return mapper.writeValueAsBytes(objectNode);
-		} catch (Exception e) {
-			throw new RuntimeException("Failed to serialize row", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java
deleted file mode 100644
index 01e72ca..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * The deserialization schema describes how to turn the byte key / value messages delivered by certain
- * data sources (for example Apache Kafka) into data types (Java/Scala objects) that are
- * processed by Flink.
- * 
- * @param <T> The type created by the keyed deserialization schema.
- */
-public interface KeyedDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
-
-	/**
-	 * Deserializes the byte message.
-	 *
-	 * @param messageKey the key as a byte array (null if no key has been set)
-	 * @param message The message, as a byte array. (null if the message was empty or deleted)
-	 * @param partition The partition the message has originated from
-	 * @param offset the offset of the message in the original source (for example the Kafka offset)  @return The deserialized message as an object.
-	 */
-	T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException;
-
-	/**
-	 * Method to decide whether the element signals the end of the stream. If
-	 * true is returned the element won't be emitted.
-	 * 
-	 * @param nextElement The element to test for the end-of-stream signal.
-	 * @return True, if the element signals end of stream, false otherwise.
-	 */
-	boolean isEndOfStream(T nextElement);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
deleted file mode 100644
index 4b9dba2..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.util.serialization;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-import java.io.IOException;
-
-/**
- * A simple wrapper for using the DeserializationSchema with the KeyedDeserializationSchema
- * interface
- * @param <T> The type created by the deserialization schema.
- */
-public class KeyedDeserializationSchemaWrapper<T> implements KeyedDeserializationSchema<T> {
-
-	private static final long serialVersionUID = 2651665280744549932L;
-
-	private final DeserializationSchema<T> deserializationSchema;
-
-	public KeyedDeserializationSchemaWrapper(DeserializationSchema<T> deserializationSchema) {
-		this.deserializationSchema = deserializationSchema;
-	}
-	@Override
-	public T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
-		return deserializationSchema.deserialize(message);
-	}
-
-	@Override
-	public boolean isEndOfStream(T nextElement) {
-		return deserializationSchema.isEndOfStream(nextElement);
-	}
-
-	@Override
-	public TypeInformation<T> getProducedType() {
-		return deserializationSchema.getProducedType();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
deleted file mode 100644
index 701281e..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.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.util.serialization;
-
-import java.io.Serializable;
-
-/**
- * The serialization schema describes how to turn a data object into a different serialized
- * representation. Most data sinks (for example Apache Kafka) require the data to be handed
- * to them in a specific format (for example as byte strings).
- * 
- * @param <T> The type to be serialized.
- */
-public interface KeyedSerializationSchema<T> extends Serializable {
-
-	/**
-	 * Serializes the key of the incoming element to a byte array
-	 * This method might return null if no key is available.
-	 *
-	 * @param element The incoming element to be serialized
-	 * @return the key of the element as a byte array
-	 */
-	byte[] serializeKey(T element);
-
-
-	/**
-	 * Serializes the value of the incoming element to a byte array
-	 * 
-	 * @param element The incoming element to be serialized
-	 * @return the value of the element as a byte array
-	 */
-	byte[] serializeValue(T element);
-
-	/**
-	 * Optional method to determine the target topic for the element
-	 *
-	 * @param element Incoming element to determine the target topic from
-	 * @return null or the target topic
-	 */
-	String getTargetTopic(T element);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
deleted file mode 100644
index 1b3e486..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.util.serialization;
-
-/**
- * A simple wrapper for using the SerializationSchema with the KeyedDeserializationSchema
- * interface
- * @param <T> The type to serialize
- */
-public class KeyedSerializationSchemaWrapper<T> implements KeyedSerializationSchema<T> {
-
-	private static final long serialVersionUID = 1351665280744549933L;
-
-	private final SerializationSchema<T> serializationSchema;
-
-	public KeyedSerializationSchemaWrapper(SerializationSchema<T> serializationSchema) {
-		this.serializationSchema = serializationSchema;
-	}
-
-	@Override
-	public byte[] serializeKey(T element) {
-		return null;
-	}
-
-	@Override
-	public byte[] serializeValue(T element) {
-		return serializationSchema.serialize(element);
-	}
-
-	@Override
-	public String getTargetTopic(T element) {
-		return null; // we are never overriding the topic
-	}
-}


[41/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
new file mode 100644
index 0000000..fc4a35e
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
@@ -0,0 +1,916 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs;
+
+import org.apache.commons.lang3.time.StopWatch;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink;
+import org.apache.flink.util.Preconditions;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Sink that emits its input elements to rolling {@link org.apache.hadoop.fs.FileSystem} files. This
+ * is integrated with the checkpointing mechanism to provide exactly once semantics.
+ *
+ * <p>
+ * When creating the sink a {@code basePath} must be specified. The base directory contains
+ * one directory for every bucket. The bucket directories themselves contain several part files.
+ * These contain the actual written data.
+ *
+ * <p>
+ * The sink uses a {@link Bucketer} to determine the name of bucket directories inside the
+ * base directory. Whenever the {@code Bucketer} returns a different directory name than
+ * it returned before the sink will close the current part files inside that bucket
+ * and start the new bucket directory. The default bucketer is a {@link DateTimeBucketer} with
+ * date format string {@code ""yyyy-MM-dd--HH"}. You can specify a custom {@code Bucketer}
+ * using {@link #setBucketer(Bucketer)}. For example, use
+ * {@link NonRollingBucketer} if you don't want to have
+ * buckets but still write part files in a fault-tolerant way.
+ *
+ * <p>
+ * The filenames of the part files contain the part prefix, the parallel subtask index of the sink
+ * and a rolling counter, for example {@code "part-1-17"}. Per default the part prefix is
+ * {@code "part"} but this can be
+ * configured using {@link #setPartPrefix(String)}. When a part file becomes bigger
+ * than the batch size the current part file is closed, the part counter is increased and
+ * a new part file is created. The batch size defaults to {@code 384MB}, this can be configured
+ * using {@link #setBatchSize(long)}.
+ *
+ * <p>
+ * Part files can be in one of three states: in-progress, pending or finished. The reason for this
+ * is how the sink works together with the checkpointing mechanism to provide exactly-once semantics
+ * and fault-tolerance. The part file that is currently being written to is in-progress. Once
+ * a part file is closed for writing it becomes pending. When a checkpoint is successful the
+ * currently pending files will be moved to finished. If a failure occurs the pending files
+ * will be deleted to reset state to the last checkpoint. The data in in-progress files will
+ * also have to be rolled back. If the {@code FileSystem} supports the {@code truncate} call
+ * this will be used to reset the file back to a previous state. If not, a special file
+ * with the same name as the part file and the suffix {@code ".valid-length"} will be written
+ * that contains the length up to which the file contains valid data. When reading the file
+ * it must be ensured that it is only read up to that point. The prefixes and suffixes for
+ * the different file states and valid-length files can be configured, for example with
+ * {@link #setPendingSuffix(String)}.
+ *
+ * <p>
+ * Note: If checkpointing is not enabled the pending files will never be moved to the finished state.
+ * In that case, the pending suffix/prefix can be set to {@code ""} to make the sink work
+ * in a non-fault-tolerant way but still provide output without prefixes and suffixes.
+ *
+ * <p>
+ * The part files are written using an instance of {@link Writer}. By default
+ * {@link org.apache.flink.streaming.connectors.fs.StringWriter} is used, which writes the result
+ * of {@code toString()} for every element. Separated by newlines. You can configure the writer
+ * using {@link #setWriter(Writer)}. For example,
+ * {@link org.apache.flink.streaming.connectors.fs.SequenceFileWriter} can be used to write
+ * Hadoop {@code SequenceFiles}.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre>{@code
+ *     new RollingSink<Tuple2<IntWritable, Text>>(outPath)
+ *         .setWriter(new SequenceFileWriter<IntWritable, Text>())
+ *         .setBucketer(new DateTimeBucketer("yyyy-MM-dd--HHmm")
+ * }</pre>
+ *
+ * This will create a sink that writes to {@code SequenceFiles} and rolls every minute.
+ *
+ * @see DateTimeBucketer
+ * @see StringWriter
+ * @see SequenceFileWriter
+ *
+ * @param <T> Type of the elements emitted by this sink
+ *
+ * @deprecated use {@link BucketingSink} instead.
+ */
+@Deprecated
+public class RollingSink<T> extends RichSinkFunction<T>
+		implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener {
+
+	private static final long serialVersionUID = 1L;
+
+	private static Logger LOG = LoggerFactory.getLogger(RollingSink.class);
+
+
+	// --------------------------------------------------------------------------------------------
+	//  User configuration values
+	// --------------------------------------------------------------------------------------------
+	// These are initialized with some defaults but are meant to be changeable by the user
+
+	/**
+	 * The default maximum size of part files (currently {@code 384 MB}).
+	 */
+	private final long DEFAULT_BATCH_SIZE = 1024L * 1024L * 384L;
+
+	/**
+	 * This is used for part files that we are writing to but which where not yet confirmed
+	 * by a checkpoint.
+	 */
+	private final String DEFAULT_IN_PROGRESS_SUFFIX = ".in-progress";
+
+	/**
+	 * See above, but for prefix
+	 */
+	private final String DEFAULT_IN_PROGRESS_PREFIX = "_";
+
+	/**
+	 * This is used for part files that we are not writing to but which are not yet confirmed by
+	 * checkpoint.
+	 */
+	private final String DEFAULT_PENDING_SUFFIX = ".pending";
+
+	/**
+	 * See above, but for prefix.
+	 */
+	private final String DEFAULT_PENDING_PREFIX = "_";
+
+	/**
+	 * When truncate() is not supported on the used FileSystem we instead write a
+	 * file along the part file with this ending that contains the length up to which
+	 * the part file is valid.
+	 */
+	private final String DEFAULT_VALID_SUFFIX = ".valid-length";
+
+	/**
+	 * See above, but for prefix.
+	 */
+	private final String DEFAULT_VALID_PREFIX = "_";
+
+	/**
+	 * The default prefix for part files.
+	 */
+	private final String DEFAULT_PART_REFIX = "part";
+
+	/**
+	 * The default timeout for asynchronous operations such as recoverLease and truncate. In
+	 * milliseconds.
+	 */
+	private final long DEFAULT_ASYNC_TIMEOUT_MS = 60 * 1000;
+
+
+	/**
+	 * The base {@code Path} that stores all bucket directories.
+	 */
+	private final String basePath;
+
+	/**
+	 * The {@code Bucketer} that is used to determine the path of bucket directories.
+	 */
+	private Bucketer bucketer;
+
+	/**
+	 * We have a template and call duplicate() for each parallel writer in open() to get the actual
+	 * writer that is used for the part files.
+	 */
+	private Writer<T> writerTemplate;
+
+	/**
+	 * The actual writer that we user for writing the part files.
+	 */
+	private Writer<T> writer;
+
+	/**
+	 * Maximum size of part files. If files exceed this we close and create a new one in the same
+	 * bucket directory.
+	 */
+	private long batchSize;
+
+	// These are the actually configured prefixes/suffixes
+	private String inProgressSuffix = DEFAULT_IN_PROGRESS_SUFFIX;
+	private String inProgressPrefix = DEFAULT_IN_PROGRESS_PREFIX;
+
+	private String pendingSuffix = DEFAULT_PENDING_SUFFIX;
+	private String pendingPrefix = DEFAULT_PENDING_PREFIX;
+
+	private String validLengthSuffix = DEFAULT_VALID_SUFFIX;
+	private String validLengthPrefix= DEFAULT_VALID_PREFIX;
+
+	private String partPrefix = DEFAULT_PART_REFIX;
+
+	/**
+	 * The timeout for asynchronous operations such as recoverLease and truncate. In
+	 * milliseconds.
+	 */
+	private long asyncTimeout = DEFAULT_ASYNC_TIMEOUT_MS;
+
+	// --------------------------------------------------------------------------------------------
+	//  Internal fields (not configurable by user)
+	// --------------------------------------------------------------------------------------------
+
+
+	/**
+	 * The part file that we are currently writing to.
+	 */
+	private transient Path currentPartPath;
+
+	/**
+	 * The bucket directory that we are currently filling.
+	 */
+	private transient Path currentBucketDirectory;
+
+	/**
+	 * For counting the part files inside a bucket directory. Part files follow the patter
+	 * {@code "{part-prefix}-{subtask}-{count}"}. When creating new part files we increase the counter.
+	 */
+	private transient int partCounter;
+
+	/**
+	 * Tracks if the writer is currently opened or closed.
+	 */
+	private transient boolean isWriterOpen;
+
+	/**
+	 * We use reflection to get the .truncate() method, this is only available starting with
+	 * Hadoop 2.7
+	 */
+	private transient Method refTruncate;
+
+	/**
+	 * The state object that is handled by flink from snapshot/restore. In there we store the
+	 * current part file path, the valid length of the in-progress files and pending part files.
+	 */
+	private transient BucketState bucketState;
+
+	private transient ListState<BucketState> restoredBucketStates;
+
+	/**
+	 * User-defined FileSystem parameters.
+     */
+	private Configuration fsConfig;
+
+	/**
+	 * The FileSystem reference.
+	 */
+	private transient FileSystem fs;
+	/**
+	 * Creates a new {@code RollingSink} that writes files to the given base directory.
+	 *
+	 * <p>
+	 * This uses a{@link DateTimeBucketer} as bucketer and a {@link StringWriter} has writer.
+	 * The maximum bucket size is set to 384 MB.
+	 *
+	 * @param basePath The directory to which to write the bucket files.
+	 */
+	public RollingSink(String basePath) {
+		this.basePath = basePath;
+		this.bucketer = new DateTimeBucketer();
+		this.batchSize = DEFAULT_BATCH_SIZE;
+		this.writerTemplate = new StringWriter<>();
+	}
+
+	/**
+	 * Specify a custom {@code Configuration} that will be used when creating
+	 * the {@link FileSystem} for writing.
+	 */
+	public RollingSink<T> setFSConfig(Configuration config) {
+		this.fsConfig = new Configuration();
+		fsConfig.addAll(config);
+		return this;
+	}
+
+	/**
+	 * Specify a custom {@code Configuration} that will be used when creating
+	 * the {@link FileSystem} for writing.
+	 */
+	public RollingSink<T> setFSConfig(org.apache.hadoop.conf.Configuration config) {
+		this.fsConfig = new Configuration();
+		for(Map.Entry<String, String> entry : config) {
+			fsConfig.setString(entry.getKey(), entry.getValue());
+		}
+		return this;
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
+		if (this.writerTemplate instanceof InputTypeConfigurable) {
+			((InputTypeConfigurable) writerTemplate).setInputType(type, executionConfig);
+		}
+	}
+
+	@Override
+	public void initializeState(FunctionInitializationContext context) throws Exception {
+		Preconditions.checkArgument(this.restoredBucketStates == null,
+			"The " + getClass().getSimpleName() + " has already been initialized.");
+
+		initFileSystem();
+
+		if (this.refTruncate == null) {
+			this.refTruncate = reflectTruncate(fs);
+		}
+
+		OperatorStateStore stateStore = context.getOperatorStateStore();
+		restoredBucketStates = stateStore.getSerializableListState("rolling-states");
+
+		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+		if (context.isRestored()) {
+			LOG.info("Restoring state for the {} (taskIdx={}).", getClass().getSimpleName(), subtaskIndex);
+
+			for (BucketState bucketState : restoredBucketStates.get()) {
+				handleRestoredBucketState(bucketState);
+			}
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("{} (taskIdx= {}) restored {}", getClass().getSimpleName(), subtaskIndex, bucketState);
+			}
+		} else {
+			LOG.info("No state to restore for the {} (taskIdx= {}).", getClass().getSimpleName(), subtaskIndex);
+		}
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+
+		partCounter = 0;
+
+		this.writer = writerTemplate.duplicate();
+
+		bucketState = new BucketState();
+	}
+
+	/**
+	 * Create a file system with the user-defined hdfs config
+	 * @throws IOException
+	 */
+	private void initFileSystem() throws IOException {
+		if (fs != null) {
+			return;
+		}
+		org.apache.hadoop.conf.Configuration hadoopConf = HadoopFileSystem.getHadoopConfiguration();
+		if (fsConfig != null) {
+			String disableCacheName = String.format("fs.%s.impl.disable.cache", new Path(basePath).toUri().getScheme());
+			hadoopConf.setBoolean(disableCacheName, true);
+			for (String key : fsConfig.keySet()) {
+				hadoopConf.set(key, fsConfig.getString(key, null));
+			}
+		}
+
+		fs = new Path(basePath).getFileSystem(hadoopConf);
+	}
+
+	@Override
+	public void close() throws Exception {
+		closeCurrentPartFile();
+	}
+
+	@Override
+	public void invoke(T value) throws Exception {
+		if (shouldRoll()) {
+			openNewPartFile();
+		}
+		writer.write(value);
+	}
+
+	/**
+	 * Determines whether we should change the bucket file we are writing to.
+	 *
+	 * <p>
+	 * This will roll if no file was created yet, if the file size is larger than the specified size
+	 * or if the {@code Bucketer} determines that we should roll.
+	 */
+	private boolean shouldRoll() throws IOException {
+		boolean shouldRoll = false;
+		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+		if (!isWriterOpen) {
+			shouldRoll = true;
+			LOG.debug("RollingSink {} starting new initial bucket. ", subtaskIndex);
+		}
+		if (bucketer.shouldStartNewBucket(new Path(basePath), currentBucketDirectory)) {
+			shouldRoll = true;
+			LOG.debug("RollingSink {} starting new bucket because {} said we should. ", subtaskIndex, bucketer);
+			// we will retrieve a new bucket base path in openNewPartFile so reset the part counter
+			partCounter = 0;
+		}
+		if (isWriterOpen) {
+			long writePosition = writer.getPos();
+			if (isWriterOpen && writePosition > batchSize) {
+				shouldRoll = true;
+				LOG.debug(
+						"RollingSink {} starting new bucket because file position {} is above batch size {}.",
+						subtaskIndex,
+						writePosition,
+						batchSize);
+			}
+		}
+		return shouldRoll;
+	}
+
+	/**
+	 * Opens a new part file.
+	 *
+	 * <p>
+	 * This closes the old bucket file and retrieves a new bucket path from the {@code Bucketer}.
+	 */
+	private void openNewPartFile() throws Exception {
+		closeCurrentPartFile();
+
+		Path newBucketDirectory = bucketer.getNextBucketPath(new Path(basePath));
+
+		if (!newBucketDirectory.equals(currentBucketDirectory)) {
+			currentBucketDirectory = newBucketDirectory;
+			try {
+				if (fs.mkdirs(currentBucketDirectory)) {
+					LOG.debug("Created new bucket directory: {}", currentBucketDirectory);
+				}
+			} catch (IOException e) {
+				throw new RuntimeException("Could not create base path for new rolling file.", e);
+			}
+		}
+
+		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+		currentPartPath = new Path(currentBucketDirectory, partPrefix + "-" + subtaskIndex + "-" + partCounter);
+
+		// This should work since there is only one parallel subtask that tries names with
+		// our subtask id. Otherwise we would run into concurrency issues here.
+		while (fs.exists(currentPartPath) ||
+				fs.exists(getPendingPathFor(currentPartPath)) ||
+				fs.exists(getInProgressPathFor(currentPartPath))) {
+			partCounter++;
+			currentPartPath = new Path(currentBucketDirectory, partPrefix + "-" + subtaskIndex + "-" + partCounter);
+		}
+
+		// increase, so we don't have to check for this name next time
+		partCounter++;
+
+		LOG.debug("Next part path is {}", currentPartPath.toString());
+
+		Path inProgressPath = getInProgressPathFor(currentPartPath);
+		writer.open(fs, inProgressPath);
+		isWriterOpen = true;
+	}
+
+	private Path getPendingPathFor(Path path) {
+		return new Path(path.getParent(), pendingPrefix + path.getName()).suffix(pendingSuffix);
+	}
+
+	private Path getInProgressPathFor(Path path) {
+		return new Path(path.getParent(), inProgressPrefix + path.getName()).suffix(inProgressSuffix);
+	}
+
+	private Path getValidLengthPathFor(Path path) {
+		return new Path(path.getParent(), validLengthPrefix + path.getName()).suffix(validLengthSuffix);
+	}
+
+	/**
+	 * Closes the current part file.
+	 *
+	 * <p>
+	 * This moves the current in-progress part file to a pending file and adds it to the list
+	 * of pending files in our bucket state.
+	 */
+	private void closeCurrentPartFile() throws Exception {
+		if (isWriterOpen) {
+			writer.close();
+			isWriterOpen = false;
+		}
+
+		if (currentPartPath != null) {
+			Path inProgressPath = getInProgressPathFor(currentPartPath);
+			Path pendingPath = getPendingPathFor(currentPartPath);
+			fs.rename(inProgressPath, pendingPath);
+			LOG.debug("Moving in-progress bucket {} to pending file {}", inProgressPath, pendingPath);
+			this.bucketState.pendingFiles.add(currentPartPath.toString());
+		}
+	}
+
+	/**
+	 * Gets the truncate() call using reflection.
+	 * <p>
+	 * <b>NOTE: </b>This code comes from Flume
+	 */
+	private Method reflectTruncate(FileSystem fs) {
+		Method m = null;
+		if (fs != null) {
+			Class<?> fsClass = fs.getClass();
+			try {
+				m = fsClass.getMethod("truncate", Path.class, long.class);
+			} catch (NoSuchMethodException ex) {
+				LOG.debug("Truncate not found. Will write a file with suffix '{}' " +
+						" and prefix '{}' to specify how many bytes in a bucket are valid.", validLengthSuffix, validLengthPrefix);
+				return null;
+			}
+
+			// verify that truncate actually works
+			FSDataOutputStream outputStream;
+			Path testPath = new Path(UUID.randomUUID().toString());
+			try {
+				outputStream = fs.create(testPath);
+				outputStream.writeUTF("hello");
+				outputStream.close();
+			} catch (IOException e) {
+				LOG.error("Could not create file for checking if truncate works.", e);
+				throw new RuntimeException("Could not create file for checking if truncate works.", e);
+			}
+
+			try {
+				m.invoke(fs, testPath, 2);
+			} catch (IllegalAccessException | InvocationTargetException e) {
+				LOG.debug("Truncate is not supported.", e);
+				m = null;
+			}
+
+			try {
+				fs.delete(testPath, false);
+			} catch (IOException e) {
+				LOG.error("Could not delete truncate test file.", e);
+				throw new RuntimeException("Could not delete truncate test file.", e);
+			}
+		}
+		return m;
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {
+		synchronized (bucketState.pendingFilesPerCheckpoint) {
+			Iterator<Map.Entry<Long, List<String>>> pendingCheckpointsIt =
+				bucketState.pendingFilesPerCheckpoint.entrySet().iterator();
+
+			while (pendingCheckpointsIt.hasNext()) {
+				Map.Entry<Long, List<String>> entry = pendingCheckpointsIt.next();
+				Long pastCheckpointId = entry.getKey();
+
+				if (pastCheckpointId <= checkpointId) {
+					LOG.debug("Moving pending files to final location for checkpoint {}", pastCheckpointId);
+					// All the pending files are buckets that have been completed but are waiting to be renamed
+					// to their final name
+					for (String filename : entry.getValue()) {
+						Path finalPath = new Path(filename);
+						Path pendingPath = getPendingPathFor(finalPath);
+
+						fs.rename(pendingPath, finalPath);
+						LOG.debug("Moving pending file {} to final location after complete checkpoint {}.",
+								pendingPath, pastCheckpointId);
+					}
+					pendingCheckpointsIt.remove();
+				}
+			}
+		}
+	}
+
+	@Override
+	public void snapshotState(FunctionSnapshotContext context) throws Exception {
+		Preconditions.checkNotNull(restoredBucketStates,
+			"The " + getClass().getSimpleName() + " has not been properly initialized.");
+
+		int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
+		
+		if (isWriterOpen) {
+			bucketState.currentFile = currentPartPath.toString();
+			bucketState.currentFileValidLength = writer.flush();
+		}
+
+		synchronized (bucketState.pendingFilesPerCheckpoint) {
+			bucketState.pendingFilesPerCheckpoint.put(context.getCheckpointId(), bucketState.pendingFiles);
+		}
+		bucketState.pendingFiles = new ArrayList<>();
+
+		restoredBucketStates.clear();
+		restoredBucketStates.add(bucketState);
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("{} (taskIdx={}) checkpointed {}.", getClass().getSimpleName(), subtaskIdx, bucketState);
+		}
+	}
+
+	private void handleRestoredBucketState(BucketState bucketState) {
+		// we can clean all the pending files since they were renamed to
+		// final files after this checkpoint was successful
+		// (we re-start from the last **successful** checkpoint)
+		bucketState.pendingFiles.clear();
+
+		if (bucketState.currentFile != null) {
+			// We were writing to a file when the last checkpoint occurred. This file can either
+			// be still in-progress or became a pending file at some point after the checkpoint.
+			// Either way, we have to truncate it back to a valid state (or write a .valid-length
+			// file that specifies up to which length it is valid) and rename it to the final name
+			// before starting a new bucket file.
+			Path partPath = new Path(bucketState.currentFile);
+			try {
+				Path partPendingPath = getPendingPathFor(partPath);
+				Path partInProgressPath = getInProgressPathFor(partPath);
+
+				if (fs.exists(partPendingPath)) {
+					LOG.debug("In-progress file {} has been moved to pending after checkpoint, moving to final location.", partPath);
+					// has been moved to pending in the mean time, rename to final location
+					fs.rename(partPendingPath, partPath);
+				} else if (fs.exists(partInProgressPath)) {
+					LOG.debug("In-progress file {} is still in-progress, moving to final location.", partPath);
+					// it was still in progress, rename to final path
+					fs.rename(partInProgressPath, partPath);
+				} else if (fs.exists(partPath)) {
+					LOG.debug("In-Progress file {} was already moved to final location {}.", bucketState.currentFile, partPath);
+				} else {
+					LOG.debug("In-Progress file {} was neither moved to pending nor is still in progress. Possibly, " +
+							"it was moved to final location by a previous snapshot restore", bucketState.currentFile);
+				}
+
+				if (this.refTruncate == null) {
+					this.refTruncate = reflectTruncate(fs);
+				}
+
+				// truncate it or write a ".valid-length" file to specify up to which point it is valid
+				if (refTruncate != null) {
+					LOG.debug("Truncating {} to valid length {}", partPath, bucketState.currentFileValidLength);
+					// some-one else might still hold the lease from a previous try, we are
+					// recovering, after all ...
+					if (fs instanceof DistributedFileSystem) {
+						DistributedFileSystem dfs = (DistributedFileSystem) fs;
+						LOG.debug("Trying to recover file lease {}", partPath);
+						dfs.recoverLease(partPath);
+						boolean isclosed= dfs.isFileClosed(partPath);
+						StopWatch sw = new StopWatch();
+						sw.start();
+						while(!isclosed) {
+							if(sw.getTime() > asyncTimeout) {
+								break;
+							}
+							try {
+								Thread.sleep(500);
+							} catch (InterruptedException e1) {
+								// ignore it
+							}
+							isclosed = dfs.isFileClosed(partPath);
+						}
+					}
+					Boolean truncated = (Boolean) refTruncate.invoke(fs, partPath, bucketState.currentFileValidLength);
+					if (!truncated) {
+						LOG.debug("Truncate did not immediately complete for {}, waiting...", partPath);
+
+						// we must wait for the asynchronous truncate operation to complete
+						StopWatch sw = new StopWatch();
+						sw.start();
+						long newLen = fs.getFileStatus(partPath).getLen();
+						while(newLen != bucketState.currentFileValidLength) {
+							if(sw.getTime() > asyncTimeout) {
+								break;
+							}
+							try {
+								Thread.sleep(500);
+							} catch (InterruptedException e1) {
+								// ignore it
+							}
+							newLen = fs.getFileStatus(partPath).getLen();
+						}
+						if (newLen != bucketState.currentFileValidLength) {
+							throw new RuntimeException("Truncate did not truncate to right length. Should be " + bucketState.currentFileValidLength + " is " + newLen + ".");
+						}
+					}
+
+				} else {
+					LOG.debug("Writing valid-length file for {} to specify valid length {}", partPath, bucketState.currentFileValidLength);
+					Path validLengthFilePath = getValidLengthPathFor(partPath);
+					if (!fs.exists(validLengthFilePath)) {
+						FSDataOutputStream lengthFileOut = fs.create(validLengthFilePath);
+						lengthFileOut.writeUTF(Long.toString(bucketState.currentFileValidLength));
+						lengthFileOut.close();
+					}
+				}
+
+				// invalidate in the state object
+				bucketState.currentFile = null;
+				bucketState.currentFileValidLength = -1;
+				isWriterOpen = false;
+			} catch (IOException e) {
+				LOG.error("Error while restoring RollingSink state.", e);
+				throw new RuntimeException("Error while restoring RollingSink state.", e);
+			} catch (InvocationTargetException | IllegalAccessException e) {
+				LOG.error("Could not invoke truncate.", e);
+				throw new RuntimeException("Could not invoke truncate.", e);
+			}
+		}
+
+		// Move files that are confirmed by a checkpoint but did not get moved to final location
+		// because the checkpoint notification did not happen before a failure
+
+		Set<Long> pastCheckpointIds = bucketState.pendingFilesPerCheckpoint.keySet();
+		LOG.debug("Moving pending files to final location on restore.");
+		for (Long pastCheckpointId : pastCheckpointIds) {
+			// All the pending files are buckets that have been completed but are waiting to be renamed
+			// to their final name
+			for (String filename : bucketState.pendingFilesPerCheckpoint.get(pastCheckpointId)) {
+				Path finalPath = new Path(filename);
+				Path pendingPath = getPendingPathFor(finalPath);
+
+				try {
+					if (fs.exists(pendingPath)) {
+						LOG.debug("(RESTORE) Moving pending file {} to final location after complete checkpoint {}.", pendingPath, pastCheckpointId);
+						fs.rename(pendingPath, finalPath);
+					}
+				} catch (IOException e) {
+					LOG.error("(RESTORE) Error while renaming pending file {} to final path {}: {}", pendingPath, finalPath, e);
+					throw new RuntimeException("Error while renaming pending file " + pendingPath+ " to final path " + finalPath, e);
+				}
+			}
+		}
+
+		synchronized (bucketState.pendingFilesPerCheckpoint) {
+			bucketState.pendingFilesPerCheckpoint.clear();
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Setters for User configuration values
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Sets the maximum bucket size in bytes.
+	 *
+	 * <p>
+	 * When a bucket part file becomes larger than this size a new bucket part file is started and
+	 * the old one is closed. The name of the bucket files depends on the {@link Bucketer}.
+	 *
+	 * @param batchSize The bucket part file size in bytes.
+	 */
+	public RollingSink<T> setBatchSize(long batchSize) {
+		this.batchSize = batchSize;
+		return this;
+	}
+
+	/**
+	 * Sets the {@link Bucketer} to use for determining the bucket files to write to.
+	 *
+	 * @param bucketer The bucketer to use.
+	 */
+	public RollingSink<T> setBucketer(Bucketer bucketer) {
+		this.bucketer = bucketer;
+		return this;
+	}
+
+	/**
+	 * Sets the {@link Writer} to be used for writing the incoming elements to bucket files.
+	 *
+	 * @param writer The {@code Writer} to use.
+	 */
+	public RollingSink<T> setWriter(Writer<T> writer) {
+		this.writerTemplate = writer;
+		return this;
+	}
+
+	/**
+	 * Sets the suffix of in-progress part files. The default is {@code "in-progress"}.
+	 */
+	public RollingSink<T> setInProgressSuffix(String inProgressSuffix) {
+		this.inProgressSuffix = inProgressSuffix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of in-progress part files. The default is {@code "_"}.
+	 */
+	public RollingSink<T> setInProgressPrefix(String inProgressPrefix) {
+		this.inProgressPrefix = inProgressPrefix;
+		return this;
+	}
+
+	/**
+	 * Sets the suffix of pending part files. The default is {@code ".pending"}.
+	 */
+	public RollingSink<T> setPendingSuffix(String pendingSuffix) {
+		this.pendingSuffix = pendingSuffix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of pending part files. The default is {@code "_"}.
+	 */
+	public RollingSink<T> setPendingPrefix(String pendingPrefix) {
+		this.pendingPrefix = pendingPrefix;
+		return this;
+	}
+
+	/**
+	 * Sets the suffix of valid-length files. The default is {@code ".valid-length"}.
+	 */
+	public RollingSink<T> setValidLengthSuffix(String validLengthSuffix) {
+		this.validLengthSuffix = validLengthSuffix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of valid-length files. The default is {@code "_"}.
+	 */
+	public RollingSink<T> setValidLengthPrefix(String validLengthPrefix) {
+		this.validLengthPrefix = validLengthPrefix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of part files.  The default is {@code "part"}.
+	 */
+	public RollingSink<T> setPartPrefix(String partPrefix) {
+		this.partPrefix = partPrefix;
+		return this;
+	}
+
+	/**
+	 * Disable cleanup of leftover in-progress/pending files when the sink is opened.
+	 *
+	 * <p>
+	 * This should only be disabled if using the sink without checkpoints, to not remove
+	 * the files already in the directory.
+	 *
+	 * @deprecated This option is deprecated and remains only for backwards compatibility.
+	 * We do not clean up lingering files anymore.
+	 */
+	@Deprecated
+	public RollingSink<T> disableCleanupOnOpen() {
+		return this;
+	}
+
+	/**
+	 * Sets the default timeout for asynchronous operations such as recoverLease and truncate.
+	 *
+	 * @param timeout The timeout, in milliseconds.
+	 */
+	public RollingSink<T> setAsyncTimeout(long timeout) {
+		this.asyncTimeout = timeout;
+		return this;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Internal Classes
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * This is used for keeping track of the current in-progress files and files that we mark
+	 * for moving from pending to final location after we get a checkpoint-complete notification.
+	 */
+	static final class BucketState implements Serializable {
+		private static final long serialVersionUID = 1L;
+
+		/**
+		 * The file that was in-progress when the last checkpoint occurred.
+		 */
+		String currentFile;
+
+		/**
+		 * The valid length of the in-progress file at the time of the last checkpoint.
+		 */
+		long currentFileValidLength = -1;
+
+		/**
+		 * Pending files that accumulated since the last checkpoint.
+		 */
+		List<String> pendingFiles = new ArrayList<>();
+
+		/**
+		 * When doing a checkpoint we move the pending files since the last checkpoint to this map
+		 * with the id of the checkpoint. When we get the checkpoint-complete notification we move
+		 * pending files of completed checkpoints to their final location.
+		 */
+		final Map<Long, List<String>> pendingFilesPerCheckpoint = new HashMap<>();
+
+		@Override
+		public String toString() {
+			return
+				"In-progress=" + currentFile +
+				" validLength=" + currentFileValidLength +
+				" pendingForNextCheckpoint=" + pendingFiles +
+				" pendingForPrevCheckpoints=" + pendingFilesPerCheckpoint;
+		}
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StreamWriterBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StreamWriterBase.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StreamWriterBase.java
new file mode 100644
index 0000000..140246f
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StreamWriterBase.java
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs;
+
+import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+/**
+ * Base class for {@link Writer Writers} that write to a {@link FSDataOutputStream}.
+ */
+public abstract class StreamWriterBase<T> implements Writer<T> {
+
+	private static Logger LOG = LoggerFactory.getLogger(BucketingSink.class);
+
+	/**
+	 * The {@code FSDataOutputStream} for the current part file.
+	 */
+	private transient FSDataOutputStream outStream;
+
+	/**
+	 * We use reflection to get the hflush method or use sync as a fallback.
+	 * The idea for this and the code comes from the Flume HDFS Sink.
+	 */
+	private transient Method refHflushOrSync;
+
+	/**
+	 * Returns the current output stream, if the stream is open.
+	 */
+	protected FSDataOutputStream getStream() {
+		if (outStream == null) {
+			throw new IllegalStateException("Output stream has not been opened");
+		}
+		return outStream;
+	}
+
+	/**
+	 * If hflush is available in this version of HDFS, then this method calls
+	 * hflush, else it calls sync.
+	 * @param os - The stream to flush/sync
+	 * @throws java.io.IOException
+	 *
+	 * <p>
+	 * Note: This code comes from Flume
+	 */
+	protected void hflushOrSync(FSDataOutputStream os) throws IOException {
+		try {
+			// At this point the refHflushOrSync cannot be null,
+			// since register method would have thrown if it was.
+			this.refHflushOrSync.invoke(os);
+		} catch (InvocationTargetException e) {
+			String msg = "Error while trying to hflushOrSync!";
+			LOG.error(msg + " " + e.getCause());
+			Throwable cause = e.getCause();
+			if(cause != null && cause instanceof IOException) {
+				throw (IOException)cause;
+			}
+			throw new RuntimeException(msg, e);
+		} catch (Exception e) {
+			String msg = "Error while trying to hflushOrSync!";
+			LOG.error(msg + " " + e);
+			throw new RuntimeException(msg, e);
+		}
+	}
+
+	/**
+	 * Gets the hflush call using reflection. Fallback to sync if hflush is not available.
+	 *
+	 * <p>
+	 * Note: This code comes from Flume
+	 */
+	private Method reflectHflushOrSync(FSDataOutputStream os) {
+		Method m = null;
+		if(os != null) {
+			Class<?> fsDataOutputStreamClass = os.getClass();
+			try {
+				m = fsDataOutputStreamClass.getMethod("hflush");
+			} catch (NoSuchMethodException ex) {
+				LOG.debug("HFlush not found. Will use sync() instead");
+				try {
+					m = fsDataOutputStreamClass.getMethod("sync");
+				} catch (Exception ex1) {
+					String msg = "Neither hflush not sync were found. That seems to be " +
+							"a problem!";
+					LOG.error(msg);
+					throw new RuntimeException(msg, ex1);
+				}
+			}
+		}
+		return m;
+	}
+
+	@Override
+	public void open(FileSystem fs, Path path) throws IOException {
+		if (outStream != null) {
+			throw new IllegalStateException("Writer has already been opened");
+		}
+		outStream = fs.create(path, false);
+		if (refHflushOrSync == null) {
+			refHflushOrSync = reflectHflushOrSync(outStream);
+		}
+	}
+
+	@Override
+	public long flush() throws IOException {
+		if (outStream == null) {
+			throw new IllegalStateException("Writer is not open");
+		}
+		hflushOrSync(outStream);
+		return outStream.getPos();
+	}
+
+	@Override
+	public long getPos() throws IOException {
+		if (outStream == null) {
+			throw new IllegalStateException("Writer is not open");
+		}
+		return outStream.getPos();
+	}
+
+	@Override
+	public void close() throws IOException {
+		if (outStream != null) {
+			flush();
+			outStream.close();
+			outStream = null;
+		}
+	}
+
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
new file mode 100644
index 0000000..c3b4cb6
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.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.fs;
+
+import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * An implementation of {@code Writer} is used in conjunction with a
+ * {@link BucketingSink} to perform the actual
+ * writing to the bucket files.
+ *
+ * @param <T> The type of the elements that are being written by the sink.
+ */
+public interface Writer<T> extends Serializable {
+
+	/**
+	 * Initializes the {@code Writer} for a newly opened bucket file.
+	 * Any internal per-bucket initialization should be performed here.
+	 *
+	 * @param fs The {@link org.apache.hadoop.fs.FileSystem} containing the newly opened file.
+	 * @param path The {@link org.apache.hadoop.fs.Path} of the newly opened file.
+	 */
+	void open(FileSystem fs, Path path) throws IOException;
+
+	/**
+	 * Flushes out any internally held data, and returns the offset that the file
+	 * must be truncated to at recovery.
+	 */
+	long flush() throws IOException;
+
+	/**
+	 * Retrieves the current position, and thus size, of the output file.
+	 */
+	long getPos() throws IOException;
+
+	/**
+	 * Closes the {@code Writer}. If the writer is already closed, no action will be
+	 * taken. The call should close all state related to the current output file,
+	 * including the output stream opened in {@code open}.
+	 */
+	void close() throws IOException ;
+
+	/**
+	 * Writes one element to the bucket file.
+	 */
+	void write(T element)throws IOException;
+
+	/**
+	 * Duplicates the {@code Writer}. This is used to get one {@code Writer} for each
+	 * parallel instance of the sink.
+	 */
+	Writer<T> duplicate();
+}

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

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


[20/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/TableInputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/TableInputFormatITCase.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/TableInputFormatITCase.java
new file mode 100644
index 0000000..3dddd88
--- /dev/null
+++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/TableInputFormatITCase.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.addons.hbase;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TableInputFormatITCase extends HBaseTestingClusterAutostarter {
+	private static final String TEST_TABLE_NAME = "TableInputFormatTestTable";
+	private static final byte[] TEST_TABLE_FAMILY_NAME = "F".getBytes();
+	private static final byte[] TEST_TABLE_COLUMN_NAME = "Col".getBytes();
+
+	// These are the row ids AND also the values we will put in the test table
+	private static final String[] ROW_IDS = {"000", "111", "222", "333", "444", "555", "666", "777", "888", "999"};
+
+	@BeforeClass
+	public static void activateHBaseCluster(){
+		registerHBaseMiniClusterInClasspath();
+	}
+
+	@Before
+	public void createTestTable() throws IOException {
+		TableName tableName = TableName.valueOf(TEST_TABLE_NAME);
+		byte[][] splitKeys = {"0".getBytes(), "3".getBytes(), "6".getBytes(), "9".getBytes()};
+		createTable(tableName, TEST_TABLE_FAMILY_NAME, splitKeys);
+		HTable table = openTable(tableName);
+
+		for (String rowId : ROW_IDS) {
+			byte[] rowIdBytes = rowId.getBytes();
+			Put p = new Put(rowIdBytes);
+			// Use the rowId as the value to facilitate the testing better
+			p.add(TEST_TABLE_FAMILY_NAME, TEST_TABLE_COLUMN_NAME, rowIdBytes);
+			table.put(p);
+		}
+
+		table.close();
+	}
+
+	class InputFormatForTestTable extends TableInputFormat<Tuple1<String>> {
+		@Override
+		protected Scan getScanner() {
+			return new Scan();
+		}
+
+		@Override
+		protected String getTableName() {
+			return TEST_TABLE_NAME;
+		}
+
+		@Override
+		protected Tuple1<String> mapResultToTuple(Result r) {
+			return new Tuple1<>(new String(r.getValue(TEST_TABLE_FAMILY_NAME, TEST_TABLE_COLUMN_NAME)));
+		}
+	}
+
+	@Test
+	public void testTableInputFormat() {
+		ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment();
+		environment.setParallelism(1);
+
+		DataSet<String> resultDataSet =
+			environment.createInput(new InputFormatForTestTable()).map(new MapFunction<Tuple1<String>, String>() {
+				@Override
+				public String map(Tuple1<String> value) throws Exception {
+					return value.f0;
+				}
+			});
+
+		List<String> resultSet = new ArrayList<>();
+		resultDataSet.output(new LocalCollectionOutputFormat<>(resultSet));
+
+		try {
+			environment.execute("HBase InputFormat Test");
+		} catch (Exception e) {
+			Assert.fail("HBase InputFormat test failed. " + e.getMessage());
+		}
+
+		for (String rowId : ROW_IDS) {
+			assertTrue("Missing rowId from table: " + rowId, resultSet.contains(rowId));
+		}
+
+		assertEquals("The number of records is wrong.", ROW_IDS.length, resultSet.size());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java
new file mode 100644
index 0000000..8579dee
--- /dev/null
+++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.addons.hbase.example;
+
+public class HBaseFlinkTestConstants {
+	
+	public static final byte[] CF_SOME = "someCf".getBytes();
+	public static final byte[] Q_SOME = "someQual".getBytes();
+	public static final String TEST_TABLE_NAME = "test-table";
+	public static final String TMP_DIR = "/tmp/test";
+	
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java
new file mode 100644
index 0000000..dccf876
--- /dev/null
+++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.addons.hbase.example;
+
+import org.apache.flink.addons.hbase.TableInputFormat;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Simple stub for HBase DataSet read
+ * 
+ * To run the test first create the test table with hbase shell.
+ * 
+ * Use the following commands:
+ * <ul>
+ *     <li>create 'test-table', 'someCf'</li>
+ *     <li>put 'test-table', '1', 'someCf:someQual', 'someString'</li>
+ *     <li>put 'test-table', '2', 'someCf:someQual', 'anotherString'</li>
+ * </ul>
+ * 
+ * The test should return just the first entry.
+ * 
+ */
+public class HBaseReadExample {
+	public static void main(String[] args) throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		@SuppressWarnings("serial")
+		DataSet<Tuple2<String, String>> hbaseDs = env.createInput(new TableInputFormat<Tuple2<String, String>>() {
+			
+				@Override
+				public String getTableName() {
+					return HBaseFlinkTestConstants.TEST_TABLE_NAME;
+				}
+
+				@Override
+				protected Scan getScanner() {
+					Scan scan = new Scan();
+					scan.addColumn(HBaseFlinkTestConstants.CF_SOME, HBaseFlinkTestConstants.Q_SOME);
+					return scan;
+				}
+
+				private Tuple2<String, String> reuse = new Tuple2<String, String>();
+				
+				@Override
+				protected Tuple2<String, String> mapResultToTuple(Result r) {
+					String key = Bytes.toString(r.getRow());
+					String val = Bytes.toString(r.getValue(HBaseFlinkTestConstants.CF_SOME, HBaseFlinkTestConstants.Q_SOME));
+					reuse.setField(key, 0);
+					reuse.setField(val, 1);
+					return reuse;
+				}
+		})
+		.filter(new FilterFunction<Tuple2<String,String>>() {
+
+			@Override
+			public boolean filter(Tuple2<String, String> t) throws Exception {
+				String val = t.getField(1);
+				if(val.startsWith("someStr"))
+					return true;
+				return false;
+			}
+		});
+		
+		hbaseDs.print();
+		
+		// kick off execution.
+		env.execute();
+				
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteExample.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteExample.java
new file mode 100644
index 0000000..483bdff
--- /dev/null
+++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteExample.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.addons.hbase.example;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.Collector;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.mapreduce.TableOutputFormat;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+
+/**
+ * Simple stub for HBase DataSet write
+ * 
+ * To run the test first create the test table with hbase shell.
+ * 
+ * Use the following commands:
+ * <ul>
+ *     <li>create 'test-table', 'someCf'</li>
+ * </ul>
+ * 
+ */
+@SuppressWarnings("serial")
+public class HBaseWriteExample {
+	
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String[] args) throws Exception {
+
+		if(!parseParameters(args)) {
+			return;
+		}
+		
+		// set up the execution environment
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		// get input data
+		DataSet<String> text = getTextDataSet(env);
+		
+		DataSet<Tuple2<String, Integer>> counts = 
+				// split up the lines in pairs (2-tuples) containing: (word,1)
+				text.flatMap(new Tokenizer())
+				// group by the tuple field "0" and sum up tuple field "1"
+				.groupBy(0)
+				.sum(1);
+
+		// emit result
+		Job job = Job.getInstance();
+		job.getConfiguration().set(TableOutputFormat.OUTPUT_TABLE, outputTableName);
+		// TODO is "mapred.output.dir" really useful?
+		job.getConfiguration().set("mapred.output.dir",HBaseFlinkTestConstants.TMP_DIR);
+		counts.map(new RichMapFunction <Tuple2<String,Integer>, Tuple2<Text,Mutation>>() {
+			private transient Tuple2<Text, Mutation> reuse;
+
+			@Override
+			public void open(Configuration parameters) throws Exception {
+				super.open(parameters);
+				reuse = new Tuple2<Text, Mutation>();
+			}
+
+			@Override
+			public Tuple2<Text, Mutation> map(Tuple2<String, Integer> t) throws Exception {
+				reuse.f0 = new Text(t.f0);
+				Put put = new Put(t.f0.getBytes());
+				put.add(HBaseFlinkTestConstants.CF_SOME,HBaseFlinkTestConstants.Q_SOME, Bytes.toBytes(t.f1));
+				reuse.f1 = put;
+				return reuse;
+			}
+		}).output(new HadoopOutputFormat<Text, Mutation>(new TableOutputFormat<Text>(), job));
+		
+		// execute program
+		env.execute("WordCount (HBase sink) Example");
+	}
+	
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+	
+	/**
+	 * Implements the string tokenizer that splits sentences into words as a user-defined
+	 * FlatMapFunction. The function takes a line (String) and splits it into 
+	 * multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
+	 */
+	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
+
+		@Override
+		public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
+			// normalize and split the line
+			String[] tokens = value.toLowerCase().split("\\W+");
+			
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new Tuple2<String, Integer>(token, 1));
+				}
+			}
+		}
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputTableName = HBaseFlinkTestConstants.TEST_TABLE_NAME;
+	
+	private static boolean parseParameters(String[] args) {
+		
+		if(args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if(args.length == 2) {
+				textPath = args[0];
+				outputTableName = args[1];
+			} else {
+				System.err.println("Usage: HBaseWriteExample <text path> <output table>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing HBaseWriteExample example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from a file.");
+			System.out.println("  Usage: HBaseWriteExample <text path> <output table>");
+		}
+		return true;
+	}
+	
+	private static DataSet<String> getTextDataSet(ExecutionEnvironment env) {
+		if(fileOutput) {
+			// read the text file from given input path
+			return env.readTextFile(textPath);
+		} else {
+			// get default test text data
+			return getDefaultTextLineDataSet(env);
+		}
+	}
+	private static DataSet<String> getDefaultTextLineDataSet(ExecutionEnvironment env) {
+		return env.fromElements(WORDS);
+	}
+	private static final String[] WORDS = new String[] {
+		"To be, or not to be,--that is the question:--",
+		"Whether 'tis nobler in the mind to suffer",
+		"The slings and arrows of outrageous fortune",
+		"Or to take arms against a sea of troubles,",
+		"And by opposing end them?--To die,--to sleep,--",
+		"No more; and by a sleep to say we end",
+		"The heartache, and the thousand natural shocks",
+		"That flesh is heir to,--'tis a consummation",
+		"Devoutly to be wish'd. To die,--to sleep;--",
+		"To sleep! perchance to dream:--ay, there's the rub;",
+		"For in that sleep of death what dreams may come,",
+		"When we have shuffled off this mortal coil,",
+		"Must give us pause: there's the respect",
+		"That makes calamity of so long life;",
+		"For who would bear the whips and scorns of time,",
+		"The oppressor's wrong, the proud man's contumely,",
+		"The pangs of despis'd love, the law's delay,",
+		"The insolence of office, and the spurns",
+		"That patient merit of the unworthy takes,",
+		"When he himself might his quietus make",
+		"With a bare bodkin? who would these fardels bear,",
+		"To grunt and sweat under a weary life,",
+		"But that the dread of something after death,--",
+		"The undiscover'd country, from whose bourn",
+		"No traveller returns,--puzzles the will,",
+		"And makes us rather bear those ills we have",
+		"Than fly to others that we know not of?",
+		"Thus conscience does make cowards of us all;",
+		"And thus the native hue of resolution",
+		"Is sicklied o'er with the pale cast of thought;",
+		"And enterprises of great pith and moment,",
+		"With this regard, their currents turn awry,",
+		"And lose the name of action.--Soft you now!",
+		"The fair Ophelia!--Nymph, in thy orisons",
+		"Be all my sins remember'd."
+	};
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
new file mode 100644
index 0000000..05398db
--- /dev/null
+++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.addons.hbase.example;
+
+import java.io.IOException;
+
+import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * 
+ * This is an example how to write streams into HBase. In this example the
+ * stream will be written into a local Hbase but it is possible to adapt this
+ * example for an HBase running in a cloud. You need a running local HBase with a
+ * table "flinkExample" and a column "entry". If your HBase configuration does
+ * not fit the hbase-site.xml in the resource folder then you gave to delete temporary this
+ * hbase-site.xml to execute the example properly.
+ * 
+ */
+public class HBaseWriteStreamExample {
+
+	public static void main(String[] args) throws Exception {
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment
+				.getExecutionEnvironment();
+
+		// data stream with random numbers
+		DataStream<String> dataStream = env.addSource(new SourceFunction<String>() {
+			private static final long serialVersionUID = 1L;
+
+			private volatile boolean isRunning = true;
+
+			@Override
+			public void run(SourceContext<String> out) throws Exception {
+				while (isRunning) {
+					out.collect(String.valueOf(Math.floor(Math.random() * 100)));
+				}
+
+			}
+
+			@Override
+			public void cancel() {
+				isRunning = false;
+			}
+		});
+		dataStream.writeUsingOutputFormat(new HBaseOutputFormat());
+
+		env.execute();
+	}
+
+	/**
+	 * 
+	 * This class implements an OutputFormat for HBase
+	 *
+	 */
+	private static class HBaseOutputFormat implements OutputFormat<String> {
+
+		private org.apache.hadoop.conf.Configuration conf = null;
+		private HTable table = null;
+		private String taskNumber = null;
+		private int rowNumber = 0;
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void configure(Configuration parameters) {
+			conf = HBaseConfiguration.create();
+		}
+
+		@Override
+		public void open(int taskNumber, int numTasks) throws IOException {
+			table = new HTable(conf, "flinkExample");
+			this.taskNumber = String.valueOf(taskNumber);
+		}
+
+		@Override
+		public void writeRecord(String record) throws IOException {
+			Put put = new Put(Bytes.toBytes(taskNumber + rowNumber));
+			put.add(Bytes.toBytes("entry"), Bytes.toBytes("entry"),
+					Bytes.toBytes(rowNumber));
+			rowNumber++;
+			table.put(put);
+		}
+
+		@Override
+		public void close() throws IOException {
+			table.flushCommits();
+			table.close();
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/src/test/resources/log4j-test.properties b/flink-connectors/flink-hbase/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..804ff45
--- /dev/null
+++ b/flink-connectors/flink-hbase/src/test/resources/log4j-test.properties
@@ -0,0 +1,23 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+# http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+log4j.rootLogger=DEBUG, stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.threshold=INFO
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ABSOLUTE} %-5p %30c{1}:%4L - %m%n

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hcatalog/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hcatalog/pom.xml b/flink-connectors/flink-hcatalog/pom.xml
new file mode 100644
index 0000000..dde7996
--- /dev/null
+++ b/flink-connectors/flink-hcatalog/pom.xml
@@ -0,0 +1,182 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+	
+	<modelVersion>4.0.0</modelVersion>
+	
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-hcatalog</artifactId>
+	<name>flink-hcatalog</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hive.hcatalog</groupId>
+			<artifactId>hcatalog-core</artifactId>
+			<version>0.12.0</version>
+			<exclusions>
+				<exclusion>
+					<groupId>org.json</groupId>
+					<artifactId>json</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<dependency>
+			<groupId>org.scala-lang</groupId>
+			<artifactId>scala-library</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+	</dependencies>
+
+	<build>
+		<plugins>
+			<!-- Scala Compiler -->
+			<plugin>
+				<groupId>net.alchim31.maven</groupId>
+				<artifactId>scala-maven-plugin</artifactId>
+				<executions>
+					<!-- Run scala compiler in the process-resources phase, so that dependencies on
+						scala classes can be resolved later in the (Java) compile phase -->
+					<execution>
+						<id>scala-compile-first</id>
+						<phase>process-resources</phase>
+						<goals>
+							<goal>compile</goal>
+						</goals>
+					</execution>
+
+					<!-- Run scala compiler in the process-test-resources phase, so that dependencies on
+						 scala classes can be resolved later in the (Java) test-compile phase -->
+					<execution>
+						<id>scala-test-compile</id>
+						<phase>process-test-resources</phase>
+						<goals>
+							<goal>testCompile</goal>
+						</goals>
+					</execution>
+				</executions>
+				<configuration>
+					<jvmArgs>
+						<jvmArg>-Xms128m</jvmArg>
+						<jvmArg>-Xmx512m</jvmArg>
+					</jvmArgs>
+				</configuration>
+			</plugin>
+
+			<!-- Eclipse Integration -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-eclipse-plugin</artifactId>
+				<version>2.8</version>
+				<configuration>
+					<downloadSources>true</downloadSources>
+					<projectnatures>
+						<projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
+						<projectnature>org.eclipse.jdt.core.javanature</projectnature>
+					</projectnatures>
+					<buildcommands>
+						<buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
+					</buildcommands>
+					<classpathContainers>
+						<classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
+						<classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
+					</classpathContainers>
+					<excludes>
+						<exclude>org.scala-lang:scala-library</exclude>
+						<exclude>org.scala-lang:scala-compiler</exclude>
+					</excludes>
+					<sourceIncludes>
+						<sourceInclude>**/*.scala</sourceInclude>
+						<sourceInclude>**/*.java</sourceInclude>
+					</sourceIncludes>
+				</configuration>
+			</plugin>
+
+			<!-- Adding scala source directories to build path -->
+			<plugin>
+				<groupId>org.codehaus.mojo</groupId>
+				<artifactId>build-helper-maven-plugin</artifactId>
+				<version>1.7</version>
+				<executions>
+					<!-- Add src/main/scala to eclipse build path -->
+					<execution>
+						<id>add-source</id>
+						<phase>generate-sources</phase>
+						<goals>
+							<goal>add-source</goal>
+						</goals>
+						<configuration>
+							<sources>
+								<source>src/main/scala</source>
+							</sources>
+						</configuration>
+					</execution>
+					<!-- Add src/test/scala to eclipse build path -->
+					<execution>
+						<id>add-test-source</id>
+						<phase>generate-test-sources</phase>
+						<goals>
+							<goal>add-test-source</goal>
+						</goals>
+						<configuration>
+							<sources>
+								<source>src/test/scala</source>
+							</sources>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+
+			<!-- Scala Code Style, most of the configuration done via plugin management -->
+			<plugin>
+				<groupId>org.scalastyle</groupId>
+				<artifactId>scalastyle-maven-plugin</artifactId>
+				<configuration>
+					<configLocation>${project.basedir}/../../tools/maven/scalastyle-config.xml</configLocation>
+				</configuration>
+			</plugin>
+
+		</plugins>
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java b/flink-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java
new file mode 100644
index 0000000..859b706
--- /dev/null
+++ b/flink-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java
@@ -0,0 +1,410 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hcatalog;
+
+import org.apache.flink.api.common.io.RichInputFormat;
+import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.hadoop.mapreduce.utils.HadoopUtils;
+import org.apache.flink.api.java.hadoop.mapreduce.wrapper.HadoopInputSplit;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.WritableTypeInfo;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hive.hcatalog.common.HCatException;
+import org.apache.hive.hcatalog.common.HCatUtil;
+import org.apache.hive.hcatalog.data.DefaultHCatRecord;
+import org.apache.hive.hcatalog.data.HCatRecord;
+import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
+import org.apache.hive.hcatalog.data.schema.HCatSchema;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A InputFormat to read from HCatalog tables.
+ * The InputFormat supports projection (selection and order of fields) and partition filters.
+ *
+ * Data can be returned as {@link org.apache.hive.hcatalog.data.HCatRecord} or Flink-native tuple.
+ *
+ * Note: Flink tuples might only support a limited number of fields (depending on the API).
+ *
+ * @param <T>
+ */
+public abstract class HCatInputFormatBase<T> extends RichInputFormat<T, HadoopInputSplit> implements ResultTypeQueryable<T> {
+
+	private static final long serialVersionUID = 1L;
+
+	private Configuration configuration;
+
+	private org.apache.hive.hcatalog.mapreduce.HCatInputFormat hCatInputFormat;
+	private RecordReader<WritableComparable, HCatRecord> recordReader;
+	private boolean fetched = false;
+	private boolean hasNext;
+
+	protected String[] fieldNames = new String[0];
+	protected HCatSchema outputSchema;
+
+	private TypeInformation<T> resultType;
+
+	public HCatInputFormatBase() { }
+
+	/**
+	 * Creates a HCatInputFormat for the given database and table.
+	 * By default, the InputFormat returns {@link org.apache.hive.hcatalog.data.HCatRecord}.
+	 * The return type of the InputFormat can be changed to Flink-native tuples by calling
+	 * {@link HCatInputFormatBase#asFlinkTuples()}.
+	 *
+	 * @param database The name of the database to read from.
+	 * @param table The name of the table to read.
+	 * @throws java.io.IOException
+	 */
+	public HCatInputFormatBase(String database, String table) throws IOException {
+		this(database, table, new Configuration());
+	}
+
+	/**
+	 * Creates a HCatInputFormat for the given database, table, and
+	 * {@link org.apache.hadoop.conf.Configuration}.
+	 * By default, the InputFormat returns {@link org.apache.hive.hcatalog.data.HCatRecord}.
+	 * The return type of the InputFormat can be changed to Flink-native tuples by calling
+	 * {@link HCatInputFormatBase#asFlinkTuples()}.
+	 *
+	 * @param database The name of the database to read from.
+	 * @param table The name of the table to read.
+	 * @param config The Configuration for the InputFormat.
+	 * @throws java.io.IOException
+	 */
+	public HCatInputFormatBase(String database, String table, Configuration config) throws IOException {
+		super();
+		this.configuration = config;
+		HadoopUtils.mergeHadoopConf(this.configuration);
+
+		this.hCatInputFormat = org.apache.hive.hcatalog.mapreduce.HCatInputFormat.setInput(this.configuration, database, table);
+		this.outputSchema = org.apache.hive.hcatalog.mapreduce.HCatInputFormat.getTableSchema(this.configuration);
+
+		// configure output schema of HCatFormat
+		configuration.set("mapreduce.lib.hcat.output.schema", HCatUtil.serialize(outputSchema));
+		// set type information
+		this.resultType = new WritableTypeInfo(DefaultHCatRecord.class);
+	}
+
+	/**
+	 * Specifies the fields which are returned by the InputFormat and their order.
+	 *
+	 * @param fields The fields and their order which are returned by the InputFormat.
+	 * @return This InputFormat with specified return fields.
+	 * @throws java.io.IOException
+	 */
+	public HCatInputFormatBase<T> getFields(String... fields) throws IOException {
+
+		// build output schema
+		ArrayList<HCatFieldSchema> fieldSchemas = new ArrayList<HCatFieldSchema>(fields.length);
+		for(String field : fields) {
+			fieldSchemas.add(this.outputSchema.get(field));
+		}
+		this.outputSchema = new HCatSchema(fieldSchemas);
+
+		// update output schema configuration
+		configuration.set("mapreduce.lib.hcat.output.schema", HCatUtil.serialize(outputSchema));
+
+		return this;
+	}
+
+	/**
+	 * Specifies a SQL-like filter condition on the table's partition columns.
+	 * Filter conditions on non-partition columns are invalid.
+	 * A partition filter can significantly reduce the amount of data to be read.
+	 *
+	 * @param filter A SQL-like filter condition on the table's partition columns.
+	 * @return This InputFormat with specified partition filter.
+	 * @throws java.io.IOException
+	 */
+	public HCatInputFormatBase<T> withFilter(String filter) throws IOException {
+
+		// set filter
+		this.hCatInputFormat.setFilter(filter);
+
+		return this;
+	}
+
+	/**
+	 * Specifies that the InputFormat returns Flink tuples instead of
+	 * {@link org.apache.hive.hcatalog.data.HCatRecord}.
+	 *
+	 * Note: Flink tuples might only support a limited number of fields (depending on the API).
+	 *
+	 * @return This InputFormat.
+	 * @throws org.apache.hive.hcatalog.common.HCatException
+	 */
+	public HCatInputFormatBase<T> asFlinkTuples() throws HCatException {
+
+		// build type information
+		int numFields = outputSchema.getFields().size();
+		if(numFields > this.getMaxFlinkTupleSize()) {
+			throw new IllegalArgumentException("Only up to "+this.getMaxFlinkTupleSize()+
+					" fields can be returned as Flink tuples.");
+		}
+
+		TypeInformation[] fieldTypes = new TypeInformation[numFields];
+		fieldNames = new String[numFields];
+		for (String fieldName : outputSchema.getFieldNames()) {
+			HCatFieldSchema field = outputSchema.get(fieldName);
+
+			int fieldPos = outputSchema.getPosition(fieldName);
+			TypeInformation fieldType = getFieldType(field);
+
+			fieldTypes[fieldPos] = fieldType;
+			fieldNames[fieldPos] = fieldName;
+
+		}
+		this.resultType = new TupleTypeInfo(fieldTypes);
+
+		return this;
+	}
+
+	protected abstract int getMaxFlinkTupleSize();
+
+	private TypeInformation getFieldType(HCatFieldSchema fieldSchema) {
+
+		switch(fieldSchema.getType()) {
+			case INT:
+				return BasicTypeInfo.INT_TYPE_INFO;
+			case TINYINT:
+				return BasicTypeInfo.BYTE_TYPE_INFO;
+			case SMALLINT:
+				return BasicTypeInfo.SHORT_TYPE_INFO;
+			case BIGINT:
+				return BasicTypeInfo.LONG_TYPE_INFO;
+			case BOOLEAN:
+				return BasicTypeInfo.BOOLEAN_TYPE_INFO;
+			case FLOAT:
+				return BasicTypeInfo.FLOAT_TYPE_INFO;
+			case DOUBLE:
+				return BasicTypeInfo.DOUBLE_TYPE_INFO;
+			case STRING:
+				return BasicTypeInfo.STRING_TYPE_INFO;
+			case BINARY:
+				return PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO;
+			case ARRAY:
+				return new GenericTypeInfo(List.class);
+			case MAP:
+				return new GenericTypeInfo(Map.class);
+			case STRUCT:
+				return new GenericTypeInfo(List.class);
+			default:
+				throw new IllegalArgumentException("Unknown data type \""+fieldSchema.getType()+"\" encountered.");
+		}
+	}
+
+	/**
+	 * Returns the {@link org.apache.hadoop.conf.Configuration} of the HCatInputFormat.
+	 *
+	 * @return The Configuration of the HCatInputFormat.
+	 */
+	public Configuration getConfiguration() {
+		return this.configuration;
+	}
+
+	/**
+	 * Returns the {@link org.apache.hive.hcatalog.data.schema.HCatSchema} of the {@link org.apache.hive.hcatalog.data.HCatRecord}
+	 * returned by this InputFormat.
+	 *
+	 * @return The HCatSchema of the HCatRecords returned by this InputFormat.
+	 */
+	public HCatSchema getOutputSchema() {
+		return this.outputSchema;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  InputFormat
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void configure(org.apache.flink.configuration.Configuration parameters) {
+		// nothing to do
+	}
+
+	@Override
+	public BaseStatistics getStatistics(BaseStatistics cachedStats) throws IOException {
+		// no statistics provided at the moment
+		return null;
+	}
+
+	@Override
+	public HadoopInputSplit[] createInputSplits(int minNumSplits)
+			throws IOException {
+		configuration.setInt("mapreduce.input.fileinputformat.split.minsize", minNumSplits);
+
+		JobContext jobContext = null;
+		try {
+			jobContext = HadoopUtils.instantiateJobContext(configuration, new JobID());
+		} catch (Exception e) {
+			throw new RuntimeException(e);
+		}
+
+		List<InputSplit> splits;
+		try {
+			splits = this.hCatInputFormat.getSplits(jobContext);
+		} catch (InterruptedException e) {
+			throw new IOException("Could not get Splits.", e);
+		}
+		HadoopInputSplit[] hadoopInputSplits = new HadoopInputSplit[splits.size()];
+
+		for(int i = 0; i < hadoopInputSplits.length; i++){
+			hadoopInputSplits[i] = new HadoopInputSplit(i, splits.get(i), jobContext);
+		}
+		return hadoopInputSplits;
+	}
+
+	@Override
+	public InputSplitAssigner getInputSplitAssigner(HadoopInputSplit[] inputSplits) {
+		return new LocatableInputSplitAssigner(inputSplits);
+	}
+
+	@Override
+	public void open(HadoopInputSplit split) throws IOException {
+		TaskAttemptContext context = null;
+		try {
+			context = HadoopUtils.instantiateTaskAttemptContext(configuration, new TaskAttemptID());
+		} catch(Exception e) {
+			throw new RuntimeException(e);
+		}
+
+		try {
+			this.recordReader = this.hCatInputFormat
+					.createRecordReader(split.getHadoopInputSplit(), context);
+			this.recordReader.initialize(split.getHadoopInputSplit(), context);
+		} catch (InterruptedException e) {
+			throw new IOException("Could not create RecordReader.", e);
+		} finally {
+			this.fetched = false;
+		}
+	}
+
+	@Override
+	public boolean reachedEnd() throws IOException {
+		if(!this.fetched) {
+			fetchNext();
+		}
+		return !this.hasNext;
+	}
+
+	private void fetchNext() throws IOException {
+		try {
+			this.hasNext = this.recordReader.nextKeyValue();
+		} catch (InterruptedException e) {
+			throw new IOException("Could not fetch next KeyValue pair.", e);
+		} finally {
+			this.fetched = true;
+		}
+	}
+
+	@Override
+	public T nextRecord(T record) throws IOException {
+		if(!this.fetched) {
+			// first record
+			fetchNext();
+		}
+		if(!this.hasNext) {
+			return null;
+		}
+		try {
+
+			// get next HCatRecord
+			HCatRecord v = this.recordReader.getCurrentValue();
+			this.fetched = false;
+
+			if(this.fieldNames.length > 0) {
+				// return as Flink tuple
+				return this.buildFlinkTuple(record, v);
+
+			} else {
+				// return as HCatRecord
+				return (T)v;
+			}
+
+		} catch (InterruptedException e) {
+			throw new IOException("Could not get next record.", e);
+		}
+	}
+
+	protected abstract T buildFlinkTuple(T t, HCatRecord record) throws HCatException;
+
+	@Override
+	public void close() throws IOException {
+		this.recordReader.close();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Custom de/serialization methods
+	// --------------------------------------------------------------------------------------------
+
+	private void writeObject(ObjectOutputStream out) throws IOException {
+		out.writeInt(this.fieldNames.length);
+		for(String fieldName : this.fieldNames) {
+			out.writeUTF(fieldName);
+		}
+		this.configuration.write(out);
+	}
+
+	@SuppressWarnings("unchecked")
+	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+		this.fieldNames = new String[in.readInt()];
+		for(int i=0; i<this.fieldNames.length; i++) {
+			this.fieldNames[i] = in.readUTF();
+		}
+
+		Configuration configuration = new Configuration();
+		configuration.readFields(in);
+
+		if(this.configuration == null) {
+			this.configuration = configuration;
+		}
+
+		this.hCatInputFormat = new org.apache.hive.hcatalog.mapreduce.HCatInputFormat();
+		this.outputSchema = (HCatSchema)HCatUtil.deserialize(this.configuration.get("mapreduce.lib.hcat.output.schema"));
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Result type business
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public TypeInformation<T> getProducedType() {
+		return this.resultType;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/java/HCatInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/java/HCatInputFormat.java b/flink-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/java/HCatInputFormat.java
new file mode 100644
index 0000000..46f3cd5
--- /dev/null
+++ b/flink-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/java/HCatInputFormat.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hcatalog.java;
+
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.hcatalog.HCatInputFormatBase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hive.hcatalog.common.HCatException;
+import org.apache.hive.hcatalog.data.HCatRecord;
+
+/**
+ * A InputFormat to read from HCatalog tables.
+ * The InputFormat supports projection (selection and order of fields) and partition filters.
+ *
+ * Data can be returned as {@link HCatRecord} or Flink {@link org.apache.flink.api.java.tuple.Tuple}.
+ * Flink tuples support only up to 25 fields.
+ *
+ * @param <T>
+ */
+public class HCatInputFormat<T> extends HCatInputFormatBase<T> {
+	private static final long serialVersionUID = 1L;
+
+	public HCatInputFormat() {}
+
+	public HCatInputFormat(String database, String table) throws Exception {
+		super(database, table);
+	}
+
+	public HCatInputFormat(String database, String table, Configuration config) throws Exception {
+		super(database, table, config);
+	}
+
+
+	@Override
+	protected int getMaxFlinkTupleSize() {
+		return 25;
+	}
+
+	@Override
+	protected T buildFlinkTuple(T t, HCatRecord record) throws HCatException {
+
+		Tuple tuple = (Tuple)t;
+
+		// Extract all fields from HCatRecord
+		for(int i=0; i < this.fieldNames.length; i++) {
+
+			// get field value
+			Object o = record.get(this.fieldNames[i], this.outputSchema);
+
+			// Set field value in Flink tuple.
+			// Partition columns are returned as String and
+			//   need to be converted to original type.
+			switch(this.outputSchema.get(i).getType()) {
+				case INT:
+					if(o instanceof String) {
+						tuple.setField(Integer.parseInt((String) o), i);
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case TINYINT:
+					if(o instanceof String) {
+						tuple.setField(Byte.parseByte((String) o), i);
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case SMALLINT:
+					if(o instanceof String) {
+						tuple.setField(Short.parseShort((String) o), i);
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case BIGINT:
+					if(o instanceof String) {
+						tuple.setField(Long.parseLong((String) o), i);
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case BOOLEAN:
+					if(o instanceof String) {
+						tuple.setField(Boolean.parseBoolean((String) o), i);
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case FLOAT:
+					if(o instanceof String) {
+						tuple.setField(Float.parseFloat((String) o), i);
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case DOUBLE:
+					if(o instanceof String) {
+						tuple.setField(Double.parseDouble((String) o), i);
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case STRING:
+					tuple.setField(o, i);
+					break;
+				case BINARY:
+					if(o instanceof String) {
+						throw new RuntimeException("Cannot handle partition keys of type BINARY.");
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case ARRAY:
+					if(o instanceof String) {
+						throw new RuntimeException("Cannot handle partition keys of type ARRAY.");
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case MAP:
+					if(o instanceof String) {
+						throw new RuntimeException("Cannot handle partition keys of type MAP.");
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				case STRUCT:
+					if(o instanceof String) {
+						throw new RuntimeException("Cannot handle partition keys of type STRUCT.");
+					} else {
+						tuple.setField(o, i);
+					}
+					break;
+				default:
+					throw new RuntimeException("Invalid Type");
+			}
+		}
+
+		return (T)tuple;
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hcatalog/src/main/scala/org/apache/flink/hcatalog/scala/HCatInputFormat.scala
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hcatalog/src/main/scala/org/apache/flink/hcatalog/scala/HCatInputFormat.scala b/flink-connectors/flink-hcatalog/src/main/scala/org/apache/flink/hcatalog/scala/HCatInputFormat.scala
new file mode 100644
index 0000000..0299ee1
--- /dev/null
+++ b/flink-connectors/flink-hcatalog/src/main/scala/org/apache/flink/hcatalog/scala/HCatInputFormat.scala
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hcatalog.scala
+
+import org.apache.flink.configuration
+import org.apache.flink.hcatalog.HCatInputFormatBase
+import org.apache.hadoop.conf.Configuration
+import org.apache.hive.hcatalog.data.HCatRecord
+import org.apache.hive.hcatalog.data.schema.HCatFieldSchema
+
+/**
+ * A InputFormat to read from HCatalog tables.
+ * The InputFormat supports projection (selection and order of fields) and partition filters.
+ *
+ * Data can be returned as [[HCatRecord]] or Scala tuples.
+ * Scala tuples support only up to 22 fields.
+ *
+ */
+class HCatInputFormat[T](
+                        database: String,
+                        table: String,
+                        config: Configuration
+                          ) extends HCatInputFormatBase[T](database, table, config) {
+
+  def this(database: String, table: String) {
+    this(database, table, new Configuration)
+  }
+
+  var vals: Array[Any] = Array[Any]()
+
+  override def configure(parameters: configuration.Configuration): Unit = {
+    super.configure(parameters)
+    vals = new Array[Any](fieldNames.length)
+  }
+
+  override protected def getMaxFlinkTupleSize: Int = 22
+
+  override protected def buildFlinkTuple(t: T, record: HCatRecord): T = {
+
+    // Extract all fields from HCatRecord
+    var i: Int = 0
+    while (i < this.fieldNames.length) {
+
+        val o: AnyRef = record.get(this.fieldNames(i), this.outputSchema)
+
+        // partition columns are returned as String
+        //   Check and convert to actual type.
+        this.outputSchema.get(i).getType match {
+          case HCatFieldSchema.Type.INT =>
+            if (o.isInstanceOf[String]) {
+              vals(i) = o.asInstanceOf[String].toInt
+            }
+            else {
+              vals(i) = o.asInstanceOf[Int]
+            }
+          case HCatFieldSchema.Type.TINYINT =>
+            if (o.isInstanceOf[String]) {
+              vals(i) = o.asInstanceOf[String].toInt.toByte
+            }
+            else {
+              vals(i) = o.asInstanceOf[Byte]
+            }
+          case HCatFieldSchema.Type.SMALLINT =>
+            if (o.isInstanceOf[String]) {
+              vals(i) = o.asInstanceOf[String].toInt.toShort
+            }
+            else {
+              vals(i) = o.asInstanceOf[Short]
+            }
+          case HCatFieldSchema.Type.BIGINT =>
+            if (o.isInstanceOf[String]) {
+              vals(i) = o.asInstanceOf[String].toLong
+            }
+            else {
+              vals(i) = o.asInstanceOf[Long]
+            }
+          case HCatFieldSchema.Type.BOOLEAN =>
+            if (o.isInstanceOf[String]) {
+              vals(i) = o.asInstanceOf[String].toBoolean
+            }
+            else {
+              vals(i) = o.asInstanceOf[Boolean]
+            }
+          case HCatFieldSchema.Type.FLOAT =>
+            if (o.isInstanceOf[String]) {
+              vals(i) = o.asInstanceOf[String].toFloat
+            }
+            else {
+              vals(i) = o.asInstanceOf[Float]
+            }
+          case HCatFieldSchema.Type.DOUBLE =>
+            if (o.isInstanceOf[String]) {
+              vals(i) = o.asInstanceOf[String].toDouble
+            }
+            else {
+              vals(i) = o.asInstanceOf[Double]
+            }
+          case HCatFieldSchema.Type.STRING =>
+            vals(i) = o
+          case HCatFieldSchema.Type.BINARY =>
+            if (o.isInstanceOf[String]) {
+              throw new RuntimeException("Cannot handle partition keys of type BINARY.")
+            }
+            else {
+              vals(i) = o.asInstanceOf[Array[Byte]]
+            }
+          case HCatFieldSchema.Type.ARRAY =>
+            if (o.isInstanceOf[String]) {
+              throw new RuntimeException("Cannot handle partition keys of type ARRAY.")
+            }
+            else {
+              vals(i) = o.asInstanceOf[List[Object]]
+            }
+          case HCatFieldSchema.Type.MAP =>
+            if (o.isInstanceOf[String]) {
+              throw new RuntimeException("Cannot handle partition keys of type MAP.")
+            }
+            else {
+              vals(i) = o.asInstanceOf[Map[Object, Object]]
+            }
+          case HCatFieldSchema.Type.STRUCT =>
+            if (o.isInstanceOf[String]) {
+              throw new RuntimeException("Cannot handle partition keys of type STRUCT.")
+            }
+            else {
+              vals(i) = o.asInstanceOf[List[Object]]
+            }
+          case _ =>
+            throw new RuntimeException("Invalid type " + this.outputSchema.get(i).getType +
+              " encountered.")
+        }
+
+        i += 1
+      }
+    createScalaTuple(vals)
+  }
+
+  private def createScalaTuple(vals: Array[Any]): T = {
+
+    this.fieldNames.length match {
+      case 1 =>
+        new Tuple1(vals(0)).asInstanceOf[T]
+      case 2 =>
+        new Tuple2(vals(0), vals(1)).asInstanceOf[T]
+      case 3 =>
+        new Tuple3(vals(0), vals(1), vals(2)).asInstanceOf[T]
+      case 4 =>
+        new Tuple4(vals(0), vals(1), vals(2), vals(3)).asInstanceOf[T]
+      case 5 =>
+        new Tuple5(vals(0), vals(1), vals(2), vals(3), vals(4)).asInstanceOf[T]
+      case 6 =>
+        new Tuple6(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5)).asInstanceOf[T]
+      case 7 =>
+        new Tuple7(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6)).asInstanceOf[T]
+      case 8 =>
+        new Tuple8(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7))
+          .asInstanceOf[T]
+      case 9 =>
+        new Tuple9(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8)).asInstanceOf[T]
+      case 10 =>
+        new Tuple10(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9)).asInstanceOf[T]
+      case 11 =>
+        new Tuple11(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10)).asInstanceOf[T]
+      case 12 =>
+        new Tuple12(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11)).asInstanceOf[T]
+      case 13 =>
+        new Tuple13(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12)).asInstanceOf[T]
+      case 14 =>
+        new Tuple14(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13)).asInstanceOf[T]
+      case 15 =>
+        new Tuple15(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14)).asInstanceOf[T]
+      case 16 =>
+        new Tuple16(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15))
+          .asInstanceOf[T]
+      case 17 =>
+        new Tuple17(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
+          vals(16)).asInstanceOf[T]
+      case 18 =>
+        new Tuple18(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
+          vals(16), vals(17)).asInstanceOf[T]
+      case 19 =>
+        new Tuple19(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
+          vals(16), vals(17), vals(18)).asInstanceOf[T]
+      case 20 =>
+        new Tuple20(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
+          vals(16), vals(17), vals(18), vals(19)).asInstanceOf[T]
+      case 21 =>
+        new Tuple21(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
+          vals(16), vals(17), vals(18), vals(19), vals(20)).asInstanceOf[T]
+      case 22 =>
+        new Tuple22(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
+          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
+          vals(16), vals(17), vals(18), vals(19), vals(20), vals(21)).asInstanceOf[T]
+      case _ =>
+        throw new RuntimeException("Only up to 22 fields supported for Scala Tuples.")
+
+  }
+
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
new file mode 100644
index 0000000..b4246f5
--- /dev/null
+++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
@@ -0,0 +1,404 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.jdbc;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.RichInputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * InputFormat to read data from a database and generate Rows.
+ * The InputFormat has to be configured using the supplied InputFormatBuilder.
+ * A valid RowTypeInfo must be properly configured in the builder, e.g.: </br>
+ *
+ * <pre><code>
+ * TypeInformation<?>[] fieldTypes = new TypeInformation<?>[] {
+ *		BasicTypeInfo.INT_TYPE_INFO,
+ *		BasicTypeInfo.STRING_TYPE_INFO,
+ *		BasicTypeInfo.STRING_TYPE_INFO,
+ *		BasicTypeInfo.DOUBLE_TYPE_INFO,
+ *		BasicTypeInfo.INT_TYPE_INFO
+ *	};
+ *
+ * RowTypeInfo rowTypeInfo = new RowTypeInfo(fieldTypes);
+ *
+ * JDBCInputFormat jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+ *				.setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+ *				.setDBUrl("jdbc:derby:memory:ebookshop")
+ *				.setQuery("select * from books")
+ *				.setRowTypeInfo(rowTypeInfo)
+ *				.finish();
+ * </code></pre>
+ *
+ * In order to query the JDBC source in parallel, you need to provide a
+ * parameterized query template (i.e. a valid {@link PreparedStatement}) and
+ * a {@link ParameterValuesProvider} which provides binding values for the
+ * query parameters. E.g.:</br>
+ *
+ * <pre><code>
+ *
+ * Serializable[][] queryParameters = new String[2][1];
+ * queryParameters[0] = new String[]{"Kumar"};
+ * queryParameters[1] = new String[]{"Tan Ah Teck"};
+ *
+ * JDBCInputFormat jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+ *				.setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+ *				.setDBUrl("jdbc:derby:memory:ebookshop")
+ *				.setQuery("select * from books WHERE author = ?")
+ *				.setRowTypeInfo(rowTypeInfo)
+ *				.setParametersProvider(new GenericParameterValuesProvider(queryParameters))
+ *				.finish();
+ * </code></pre>
+ *
+ * @see Row
+ * @see ParameterValuesProvider
+ * @see PreparedStatement
+ * @see DriverManager
+ */
+public class JDBCInputFormat extends RichInputFormat<Row, InputSplit> implements ResultTypeQueryable {
+
+	private static final long serialVersionUID = 1L;
+	private static final Logger LOG = LoggerFactory.getLogger(JDBCInputFormat.class);
+
+	private String username;
+	private String password;
+	private String drivername;
+	private String dbURL;
+	private String queryTemplate;
+	private int resultSetType;
+	private int resultSetConcurrency;
+	private RowTypeInfo rowTypeInfo;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient ResultSet resultSet;
+
+	private boolean hasNext;
+	private Object[][] parameterValues;
+
+	public JDBCInputFormat() {
+	}
+
+	@Override
+	public RowTypeInfo getProducedType() {
+		return rowTypeInfo;
+	}
+
+	@Override
+	public void configure(Configuration parameters) {
+		//do nothing here
+	}
+
+	@Override
+	public void openInputFormat() {
+		//called once per inputFormat (on open)
+		try {
+			Class.forName(drivername);
+			if (username == null) {
+				dbConn = DriverManager.getConnection(dbURL);
+			} else {
+				dbConn = DriverManager.getConnection(dbURL, username, password);
+			}
+			statement = dbConn.prepareStatement(queryTemplate, resultSetType, resultSetConcurrency);
+		} catch (SQLException se) {
+			throw new IllegalArgumentException("open() failed." + se.getMessage(), se);
+		} catch (ClassNotFoundException cnfe) {
+			throw new IllegalArgumentException("JDBC-Class not found. - " + cnfe.getMessage(), cnfe);
+		}
+	}
+
+	@Override
+	public void closeInputFormat() {
+		//called once per inputFormat (on close)
+		try {
+			if(statement != null) {
+				statement.close();
+			}
+		} catch (SQLException se) {
+			LOG.info("Inputformat Statement couldn't be closed - " + se.getMessage());
+		} finally {
+			statement = null;
+		}
+
+		try {
+			if(dbConn != null) {
+				dbConn.close();
+			}
+		} catch (SQLException se) {
+			LOG.info("Inputformat couldn't be closed - " + se.getMessage());
+		} finally {
+			dbConn = null;
+		}
+
+		parameterValues = null;
+	}
+
+	/**
+	 * Connects to the source database and executes the query in a <b>parallel
+	 * fashion</b> if
+	 * this {@link InputFormat} is built using a parameterized query (i.e. using
+	 * a {@link PreparedStatement})
+	 * and a proper {@link ParameterValuesProvider}, in a <b>non-parallel
+	 * fashion</b> otherwise.
+	 *
+	 * @param inputSplit which is ignored if this InputFormat is executed as a
+	 *        non-parallel source,
+	 *        a "hook" to the query parameters otherwise (using its
+	 *        <i>splitNumber</i>)
+	 * @throws IOException if there's an error during the execution of the query
+	 */
+	@Override
+	public void open(InputSplit inputSplit) throws IOException {
+		try {
+			if (inputSplit != null && parameterValues != null) {
+				for (int i = 0; i < parameterValues[inputSplit.getSplitNumber()].length; i++) {
+					Object param = parameterValues[inputSplit.getSplitNumber()][i];
+					if (param instanceof String) {
+						statement.setString(i + 1, (String) param);
+					} else if (param instanceof Long) {
+						statement.setLong(i + 1, (Long) param);
+					} else if (param instanceof Integer) {
+						statement.setInt(i + 1, (Integer) param);
+					} else if (param instanceof Double) {
+						statement.setDouble(i + 1, (Double) param);
+					} else if (param instanceof Boolean) {
+						statement.setBoolean(i + 1, (Boolean) param);
+					} else if (param instanceof Float) {
+						statement.setFloat(i + 1, (Float) param);
+					} else if (param instanceof BigDecimal) {
+						statement.setBigDecimal(i + 1, (BigDecimal) param);
+					} else if (param instanceof Byte) {
+						statement.setByte(i + 1, (Byte) param);
+					} else if (param instanceof Short) {
+						statement.setShort(i + 1, (Short) param);
+					} else if (param instanceof Date) {
+						statement.setDate(i + 1, (Date) param);
+					} else if (param instanceof Time) {
+						statement.setTime(i + 1, (Time) param);
+					} else if (param instanceof Timestamp) {
+						statement.setTimestamp(i + 1, (Timestamp) param);
+					} else if (param instanceof Array) {
+						statement.setArray(i + 1, (Array) param);
+					} else {
+						//extends with other types if needed
+						throw new IllegalArgumentException("open() failed. Parameter " + i + " of type " + param.getClass() + " is not handled (yet)." );
+					}
+				}
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(String.format("Executing '%s' with parameters %s", queryTemplate, Arrays.deepToString(parameterValues[inputSplit.getSplitNumber()])));
+				}
+			}
+			resultSet = statement.executeQuery();
+			hasNext = resultSet.next();
+		} catch (SQLException se) {
+			throw new IllegalArgumentException("open() failed." + se.getMessage(), se);
+		}
+	}
+
+	/**
+	 * Closes all resources used.
+	 *
+	 * @throws IOException Indicates that a resource could not be closed.
+	 */
+	@Override
+	public void close() throws IOException {
+		if(resultSet == null) {
+			return;
+		}
+		try {
+			resultSet.close();
+		} catch (SQLException se) {
+			LOG.info("Inputformat ResultSet couldn't be closed - " + se.getMessage());
+		}
+	}
+
+	/**
+	 * Checks whether all data has been read.
+	 *
+	 * @return boolean value indication whether all data has been read.
+	 * @throws IOException
+	 */
+	@Override
+	public boolean reachedEnd() throws IOException {
+		return !hasNext;
+	}
+
+	/**
+	 * Stores the next resultSet row in a tuple
+	 *
+	 * @param row row to be reused.
+	 * @return row containing next {@link Row}
+	 * @throws java.io.IOException
+	 */
+	@Override
+	public Row nextRecord(Row row) throws IOException {
+		try {
+			if (!hasNext) {
+				return null;
+			}
+			for (int pos = 0; pos < row.productArity(); pos++) {
+				row.setField(pos, resultSet.getObject(pos + 1));
+			}
+			//update hasNext after we've read the record
+			hasNext = resultSet.next();
+			return row;
+		} catch (SQLException se) {
+			throw new IOException("Couldn't read data - " + se.getMessage(), se);
+		} catch (NullPointerException npe) {
+			throw new IOException("Couldn't access resultSet", npe);
+		}
+	}
+
+	@Override
+	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
+		return cachedStatistics;
+	}
+
+	@Override
+	public InputSplit[] createInputSplits(int minNumSplits) throws IOException {
+		if (parameterValues == null) {
+			return new GenericInputSplit[]{new GenericInputSplit(0, 1)};
+		}
+		GenericInputSplit[] ret = new GenericInputSplit[parameterValues.length];
+		for (int i = 0; i < ret.length; i++) {
+			ret[i] = new GenericInputSplit(i, ret.length);
+		}
+		return ret;
+	}
+
+	@Override
+	public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) {
+		return new DefaultInputSplitAssigner(inputSplits);
+	}
+
+	/**
+	 * A builder used to set parameters to the output format's configuration in a fluent way.
+	 * @return builder
+	 */
+	public static JDBCInputFormatBuilder buildJDBCInputFormat() {
+		return new JDBCInputFormatBuilder();
+	}
+
+	public static class JDBCInputFormatBuilder {
+		private final JDBCInputFormat format;
+
+		public JDBCInputFormatBuilder() {
+			this.format = new JDBCInputFormat();
+			//using TYPE_FORWARD_ONLY for high performance reads
+			this.format.resultSetType = ResultSet.TYPE_FORWARD_ONLY;
+			this.format.resultSetConcurrency = ResultSet.CONCUR_READ_ONLY;
+		}
+
+		public JDBCInputFormatBuilder setUsername(String username) {
+			format.username = username;
+			return this;
+		}
+
+		public JDBCInputFormatBuilder setPassword(String password) {
+			format.password = password;
+			return this;
+		}
+
+		public JDBCInputFormatBuilder setDrivername(String drivername) {
+			format.drivername = drivername;
+			return this;
+		}
+
+		public JDBCInputFormatBuilder setDBUrl(String dbURL) {
+			format.dbURL = dbURL;
+			return this;
+		}
+
+		public JDBCInputFormatBuilder setQuery(String query) {
+			format.queryTemplate = query;
+			return this;
+		}
+
+		public JDBCInputFormatBuilder setResultSetType(int resultSetType) {
+			format.resultSetType = resultSetType;
+			return this;
+		}
+
+		public JDBCInputFormatBuilder setResultSetConcurrency(int resultSetConcurrency) {
+			format.resultSetConcurrency = resultSetConcurrency;
+			return this;
+		}
+
+		public JDBCInputFormatBuilder setParametersProvider(ParameterValuesProvider parameterValuesProvider) {
+			format.parameterValues = parameterValuesProvider.getParameterValues();
+			return this;
+		}
+
+		public JDBCInputFormatBuilder setRowTypeInfo(RowTypeInfo rowTypeInfo) {
+			format.rowTypeInfo = rowTypeInfo;
+			return this;
+		}
+
+		public JDBCInputFormat finish() {
+			if (format.username == null) {
+				LOG.info("Username was not supplied separately.");
+			}
+			if (format.password == null) {
+				LOG.info("Password was not supplied separately.");
+			}
+			if (format.dbURL == null) {
+				throw new IllegalArgumentException("No database URL supplied");
+			}
+			if (format.queryTemplate == null) {
+				throw new IllegalArgumentException("No query supplied");
+			}
+			if (format.drivername == null) {
+				throw new IllegalArgumentException("No driver supplied");
+			}
+			if (format.rowTypeInfo == null) {
+				throw new IllegalArgumentException("No " + RowTypeInfo.class.getSimpleName() + " supplied");
+			}
+			if (format.parameterValues == null) {
+				LOG.debug("No input splitting configured (data will be read with parallelism 1).");
+			}
+			return format;
+		}
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java
new file mode 100644
index 0000000..da4b1ad
--- /dev/null
+++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.io.jdbc;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+import org.apache.flink.api.common.io.RichOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.configuration.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * OutputFormat to write tuples into a database.
+ * The OutputFormat has to be configured using the supplied OutputFormatBuilder.
+ * 
+ * @see Tuple
+ * @see DriverManager
+ */
+public class JDBCOutputFormat extends RichOutputFormat<Row> {
+	private static final long serialVersionUID = 1L;
+	
+	private static final Logger LOG = LoggerFactory.getLogger(JDBCOutputFormat.class);
+	
+	private String username;
+	private String password;
+	private String drivername;
+	private String dbURL;
+	private String query;
+	private int batchInterval = 5000;
+	
+	private Connection dbConn;
+	private PreparedStatement upload;
+	
+	private int batchCount = 0;
+	
+	public int[] typesArray;
+	
+	public JDBCOutputFormat() {
+	}
+	
+	@Override
+	public void configure(Configuration parameters) {
+	}
+	
+	/**
+	 * Connects to the target database and initializes the prepared statement.
+	 *
+	 * @param taskNumber The number of the parallel instance.
+	 * @throws IOException Thrown, if the output could not be opened due to an
+	 * I/O problem.
+	 */
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		try {
+			establishConnection();
+			upload = dbConn.prepareStatement(query);
+		} catch (SQLException sqe) {
+			throw new IllegalArgumentException("open() failed.", sqe);
+		} catch (ClassNotFoundException cnfe) {
+			throw new IllegalArgumentException("JDBC driver class not found.", cnfe);
+		}
+	}
+	
+	private void establishConnection() throws SQLException, ClassNotFoundException {
+		Class.forName(drivername);
+		if (username == null) {
+			dbConn = DriverManager.getConnection(dbURL);
+		} else {
+			dbConn = DriverManager.getConnection(dbURL, username, password);
+		}
+	}
+	
+	/**
+	 * Adds a record to the prepared statement.
+	 * <p>
+	 * When this method is called, the output format is guaranteed to be opened.
+	 * </p>
+	 * 
+	 * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to
+	 * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null))
+	 *
+	 * @param row The records to add to the output.
+	 * @see PreparedStatement
+	 * @throws IOException Thrown, if the records could not be added due to an I/O problem.
+	 */
+	@Override
+	public void writeRecord(Row row) throws IOException {
+
+		if (typesArray != null && typesArray.length > 0 && typesArray.length != row.productArity()) {
+			LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array...");
+		} 
+		try {
+
+			if (typesArray == null ) {
+				// no types provided
+				for (int index = 0; index < row.productArity(); index++) {
+					LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.productElement(index));
+					upload.setObject(index + 1, row.productElement(index));
+				}
+			} else {
+				// types provided
+				for (int index = 0; index < row.productArity(); index++) {
+
+					if (row.productElement(index) == null) {
+						upload.setNull(index + 1, typesArray[index]);
+					} else {
+						// casting values as suggested by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html
+						switch (typesArray[index]) {
+							case java.sql.Types.NULL:
+								upload.setNull(index + 1, typesArray[index]);
+								break;
+							case java.sql.Types.BOOLEAN:
+							case java.sql.Types.BIT:
+								upload.setBoolean(index + 1, (boolean) row.productElement(index));
+								break;
+							case java.sql.Types.CHAR:
+							case java.sql.Types.NCHAR:
+							case java.sql.Types.VARCHAR:
+							case java.sql.Types.LONGVARCHAR:
+							case java.sql.Types.LONGNVARCHAR:
+								upload.setString(index + 1, (String) row.productElement(index));
+								break;
+							case java.sql.Types.TINYINT:
+								upload.setByte(index + 1, (byte) row.productElement(index));
+								break;
+							case java.sql.Types.SMALLINT:
+								upload.setShort(index + 1, (short) row.productElement(index));
+								break;
+							case java.sql.Types.INTEGER:
+								upload.setInt(index + 1, (int) row.productElement(index));
+								break;
+							case java.sql.Types.BIGINT:
+								upload.setLong(index + 1, (long) row.productElement(index));
+								break;
+							case java.sql.Types.REAL:
+								upload.setFloat(index + 1, (float) row.productElement(index));
+								break;
+							case java.sql.Types.FLOAT:
+							case java.sql.Types.DOUBLE:
+								upload.setDouble(index + 1, (double) row.productElement(index));
+								break;
+							case java.sql.Types.DECIMAL:
+							case java.sql.Types.NUMERIC:
+								upload.setBigDecimal(index + 1, (java.math.BigDecimal) row.productElement(index));
+								break;
+							case java.sql.Types.DATE:
+								upload.setDate(index + 1, (java.sql.Date) row.productElement(index));
+								break;
+							case java.sql.Types.TIME:
+								upload.setTime(index + 1, (java.sql.Time) row.productElement(index));
+								break;
+							case java.sql.Types.TIMESTAMP:
+								upload.setTimestamp(index + 1, (java.sql.Timestamp) row.productElement(index));
+								break;
+							case java.sql.Types.BINARY:
+							case java.sql.Types.VARBINARY:
+							case java.sql.Types.LONGVARBINARY:
+								upload.setBytes(index + 1, (byte[]) row.productElement(index));
+								break;
+							default:
+								upload.setObject(index + 1, row.productElement(index));
+								LOG.warn("Unmanaged sql type (%s) for column %s. Best effort approach to set its value: %s.",
+									typesArray[index], index + 1, row.productElement(index));
+								// case java.sql.Types.SQLXML
+								// case java.sql.Types.ARRAY:
+								// case java.sql.Types.JAVA_OBJECT:
+								// case java.sql.Types.BLOB:
+								// case java.sql.Types.CLOB:
+								// case java.sql.Types.NCLOB:
+								// case java.sql.Types.DATALINK:
+								// case java.sql.Types.DISTINCT:
+								// case java.sql.Types.OTHER:
+								// case java.sql.Types.REF:
+								// case java.sql.Types.ROWID:
+								// case java.sql.Types.STRUC
+						}
+					}
+				}
+			}
+			upload.addBatch();
+			batchCount++;
+			if (batchCount >= batchInterval) {
+				upload.executeBatch();
+				batchCount = 0;
+			}
+		} catch (SQLException | IllegalArgumentException e) {
+			throw new IllegalArgumentException("writeRecord() failed", e);
+		}
+	}
+	
+	/**
+	 * Executes prepared statement and closes all resources of this instance.
+	 *
+	 * @throws IOException Thrown, if the input could not be closed properly.
+	 */
+	@Override
+	public void close() throws IOException {
+		try {
+			if (upload != null) {
+				upload.executeBatch();
+				upload.close();
+			}
+		} catch (SQLException se) {
+			LOG.info("Inputformat couldn't be closed - " + se.getMessage());
+		} finally {
+			upload = null;
+			batchCount = 0;
+		}
+		
+		try {
+			if (dbConn != null) {
+				dbConn.close();
+			}
+		} catch (SQLException se) {
+			LOG.info("Inputformat couldn't be closed - " + se.getMessage());
+		} finally {
+			dbConn = null;
+		}
+	}
+	
+	public static JDBCOutputFormatBuilder buildJDBCOutputFormat() {
+		return new JDBCOutputFormatBuilder();
+	}
+	
+	public static class JDBCOutputFormatBuilder {
+		private final JDBCOutputFormat format;
+		
+		protected JDBCOutputFormatBuilder() {
+			this.format = new JDBCOutputFormat();
+		}
+		
+		public JDBCOutputFormatBuilder setUsername(String username) {
+			format.username = username;
+			return this;
+		}
+		
+		public JDBCOutputFormatBuilder setPassword(String password) {
+			format.password = password;
+			return this;
+		}
+		
+		public JDBCOutputFormatBuilder setDrivername(String drivername) {
+			format.drivername = drivername;
+			return this;
+		}
+		
+		public JDBCOutputFormatBuilder setDBUrl(String dbURL) {
+			format.dbURL = dbURL;
+			return this;
+		}
+		
+		public JDBCOutputFormatBuilder setQuery(String query) {
+			format.query = query;
+			return this;
+		}
+		
+		public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) {
+			format.batchInterval = batchInterval;
+			return this;
+		}
+		
+		public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) {
+			format.typesArray = typesArray;
+			return this;
+		}
+		
+		/**
+		 * Finalizes the configuration and checks validity.
+		 * 
+		 * @return Configured JDBCOutputFormat
+		 */
+		public JDBCOutputFormat finish() {
+			if (format.username == null) {
+				LOG.info("Username was not supplied separately.");
+			}
+			if (format.password == null) {
+				LOG.info("Password was not supplied separately.");
+			}
+			if (format.dbURL == null) {
+				throw new IllegalArgumentException("No dababase URL supplied.");
+			}
+			if (format.query == null) {
+				throw new IllegalArgumentException("No query suplied");
+			}
+			if (format.drivername == null) {
+				throw new IllegalArgumentException("No driver supplied");
+			}
+			
+			return format;
+		}
+	}
+	
+}


[34/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java
new file mode 100644
index 0000000..c17aae6
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.TopicPartition;
+
+import java.util.List;
+
+/**
+ * The ConsumerCallBridge simply calls methods on the {@link KafkaConsumer}.
+ * 
+ * This indirection is necessary, because Kafka broke binary compatibility between 0.9 and 0.10,
+ * for example changing {@code assign(List)} to {@code assign(Collection)}.
+ * 
+ * Because of that, we need to two versions whose compiled code goes against different method signatures.
+ * Even though the source of subclasses may look identical, the byte code will be different, because they
+ * are compiled against different dependencies.
+ */
+public class KafkaConsumerCallBridge {
+
+	public void assignPartitions(KafkaConsumer<?, ?> consumer, List<TopicPartition> topicPartitions) throws Exception {
+		consumer.assign(topicPartitions);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java
new file mode 100644
index 0000000..9cfa840
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java
@@ -0,0 +1,332 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.metrics.MetricGroup;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
+import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper;
+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.clients.consumer.OffsetCommitCallback;
+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 java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The thread the runs the {@link KafkaConsumer}, connecting to the brokers and polling records.
+ * The thread pushes the data into a {@link Handover} to be picked up by the fetcher that will
+ * deserialize and emit the records.
+ * 
+ * <p><b>IMPORTANT:</b> This thread must not be interrupted when attempting to shut it down.
+ * The Kafka consumer code was found to not always handle interrupts well, and to even
+ * deadlock in certain situations.
+ * 
+ * <p>Implementation Note: This code is written to be reusable in later versions of the KafkaConsumer.
+ * Because Kafka is not maintaining binary compatibility, we use a "call bridge" as an indirection
+ * to the KafkaConsumer calls that change signature.
+ */
+public class KafkaConsumerThread extends Thread {
+
+	/** Logger for this consumer */
+	private final Logger log;
+
+	/** The handover of data and exceptions between the consumer thread and the task thread */
+	private final Handover handover;
+
+	/** The next offsets that the main thread should commit */
+	private final AtomicReference<Map<TopicPartition, OffsetAndMetadata>> nextOffsetsToCommit;
+
+	/** The configuration for the Kafka consumer */
+	private final Properties kafkaProperties;
+
+	/** The partitions that this consumer reads from */ 
+	private final KafkaTopicPartitionState<TopicPartition>[] subscribedPartitions;
+
+	/** We get this from the outside to publish metrics. **/
+	private final MetricGroup kafkaMetricGroup;
+
+	/** The indirections on KafkaConsumer methods, for cases where KafkaConsumer compatibility is broken */
+	private final KafkaConsumerCallBridge consumerCallBridge;
+
+	/** The maximum number of milliseconds to wait for a fetch batch */
+	private final long pollTimeout;
+
+	/** Flag whether to add Kafka's metrics to the Flink metrics */
+	private final boolean useMetrics;
+
+	/** Reference to the Kafka consumer, once it is created */
+	private volatile KafkaConsumer<byte[], byte[]> consumer;
+
+	/** Flag to mark the main work loop as alive */
+	private volatile boolean running;
+
+	/** Flag tracking whether the latest commit request has completed */
+	private volatile boolean commitInProgress;
+
+
+	public KafkaConsumerThread(
+			Logger log,
+			Handover handover,
+			Properties kafkaProperties,
+			KafkaTopicPartitionState<TopicPartition>[] subscribedPartitions,
+			MetricGroup kafkaMetricGroup,
+			KafkaConsumerCallBridge consumerCallBridge,
+			String threadName,
+			long pollTimeout,
+			boolean useMetrics) {
+
+		super(threadName);
+		setDaemon(true);
+
+		this.log = checkNotNull(log);
+		this.handover = checkNotNull(handover);
+		this.kafkaProperties = checkNotNull(kafkaProperties);
+		this.subscribedPartitions = checkNotNull(subscribedPartitions);
+		this.kafkaMetricGroup = checkNotNull(kafkaMetricGroup);
+		this.consumerCallBridge = checkNotNull(consumerCallBridge);
+		this.pollTimeout = pollTimeout;
+		this.useMetrics = useMetrics;
+
+		this.nextOffsetsToCommit = new AtomicReference<>();
+		this.running = true;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void run() {
+		// early exit check
+		if (!running) {
+			return;
+		}
+
+		// this is the means to talk to FlinkKafkaConsumer's main thread
+		final Handover handover = this.handover;
+
+		// 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) {
+			handover.reportError(t);
+			return;
+		}
+
+		// from here on, the consumer is guaranteed to be closed properly
+		try {
+			// The callback invoked by Kafka once an offset commit is complete
+			final OffsetCommitCallback offsetCommitCallback = new CommitCallback();
+
+			// tell the consumer which partitions to work with
+			consumerCallBridge.assignPartitions(consumer, convertKafkaPartitions(subscribedPartitions));
+
+			// register Kafka's very own metrics in Flink's metric reporters
+			if (useMetrics) {
+				// register Kafka metrics to Flink
+				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 Kafka metrics, register them
+					for (Map.Entry<MetricName, ? extends Metric> metric: metrics.entrySet()) {
+						kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue()));
+					}
+				}
+			}
+
+			// early exit check
+			if (!running) {
+				return;
+			}
+
+			// seek the consumer to the initial offsets
+			for (KafkaTopicPartitionState<TopicPartition> partition : subscribedPartitions) {
+				if (partition.isOffsetDefined()) {
+					log.info("Partition {} has restored initial offsets {} from checkpoint / savepoint; " +
+							"seeking the consumer to position {}",
+							partition.getKafkaPartitionHandle(), partition.getOffset(), partition.getOffset() + 1);
+
+					consumer.seek(partition.getKafkaPartitionHandle(), partition.getOffset() + 1);
+				}
+				else {
+					// for partitions that do not have offsets restored from a checkpoint/savepoint,
+					// we need to define our internal offset state for them using the initial offsets retrieved from Kafka
+					// by the KafkaConsumer, so that they are correctly checkpointed and committed on the next checkpoint
+
+					long fetchedOffset = consumer.position(partition.getKafkaPartitionHandle());
+
+					log.info("Partition {} has no initial offset; the consumer has position {}, " +
+							"so the initial offset will be set to {}",
+							partition.getKafkaPartitionHandle(), fetchedOffset, fetchedOffset - 1);
+
+					// the fetched offset represents the next record to process, so we need to subtract it by 1
+					partition.setOffset(fetchedOffset - 1);
+				}
+			}
+
+			// from now on, external operations may call the consumer
+			this.consumer = consumer;
+
+			// the latest bulk of records. may carry across the loop if the thread is woken up
+			// from blocking on the handover
+			ConsumerRecords<byte[], byte[]> records = null;
+
+			// main fetch loop
+			while (running) {
+
+				// check if there is something to commit
+				if (!commitInProgress) {
+					// get and reset the work-to-be committed, so we don't repeatedly commit the same
+					final Map<TopicPartition, OffsetAndMetadata> toCommit = nextOffsetsToCommit.getAndSet(null);
+
+					if (toCommit != null) {
+						log.debug("Sending async offset commit request to Kafka broker");
+
+						// also record that a commit is already in progress
+						// the order here matters! first set the flag, then send the commit command.
+						commitInProgress = true;
+						consumer.commitAsync(toCommit, offsetCommitCallback);
+					}
+				}
+
+				// get the next batch of records, unless we did not manage to hand the old batch over
+				if (records == null) {
+					try {
+						records = consumer.poll(pollTimeout);
+					}
+					catch (WakeupException we) {
+						continue;
+					}
+				}
+
+				try {
+					handover.produce(records);
+					records = null;
+				}
+				catch (Handover.WakeupException e) {
+					// fall through the loop
+				}
+			}
+			// end main fetch loop
+		}
+		catch (Throwable t) {
+			// let the main thread know and exit
+			// it may be that this exception comes because the main thread closed the handover, in
+			// which case the below reporting is irrelevant, but does not hurt either
+			handover.reportError(t);
+		}
+		finally {
+			// make sure the handover is closed if it is not already closed or has an error
+			handover.close();
+
+			// make sure the KafkaConsumer is closed
+			try {
+				consumer.close();
+			}
+			catch (Throwable t) {
+				log.warn("Error while closing Kafka consumer", t);
+			}
+		}
+	}
+
+	/**
+	 * Shuts this thread down, waking up the thread gracefully if blocked (without Thread.interrupt() calls).
+	 */
+	public void shutdown() {
+		running = false;
+
+		// We cannot call close() on the KafkaConsumer, because it will actually throw
+		// an exception if a concurrent call is in progress
+
+		// this wakes up the consumer if it is blocked handing over records
+		handover.wakeupProducer();
+
+		// this wakes up the consumer if it is blocked in a kafka poll 
+		if (consumer != null) {
+			consumer.wakeup();
+		}
+	}
+
+	/**
+	 * Tells this thread to commit a set of offsets. This method does not block, the committing
+	 * operation will happen asynchronously.
+	 * 
+	 * <p>Only one commit operation may be pending at any time. If the committing takes longer than
+	 * the frequency with which this method is called, then some commits may be skipped due to being
+	 * superseded  by newer ones.
+	 * 
+	 * @param offsetsToCommit The offsets to commit
+	 */
+	public void setOffsetsToCommit(Map<TopicPartition, OffsetAndMetadata> offsetsToCommit) {
+		// record the work to be committed by the main consumer thread and make sure the consumer notices that
+		if (nextOffsetsToCommit.getAndSet(offsetsToCommit) != null) {
+			log.warn("Committing offsets to Kafka takes longer than the checkpoint interval. " +
+					"Skipping commit of previous offsets because newer complete checkpoint offsets are available. " +
+					"This does not compromise Flink's checkpoint integrity.");
+		}
+
+		// if the consumer is blocked in a poll() or handover operation, wake it up to commit soon
+		handover.wakeupProducer();
+		if (consumer != null) {
+			consumer.wakeup();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private 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;
+	}
+
+	// ------------------------------------------------------------------------
+
+	private class CommitCallback implements OffsetCommitCallback {
+
+		@Override
+		public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception ex) {
+			commitInProgress = false;
+
+			if (ex != null) {
+				log.warn("Committing offsets to Kafka failed. This does not compromise Flink's checkpoints.", ex);
+			}
+		}
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java
new file mode 100644
index 0000000..7a82365
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java
@@ -0,0 +1,482 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.core.testutils.MultiShotLatch;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.kafka.internal.Handover;
+import org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher;
+import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+
+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.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.common.TopicPartition;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.mockito.Mockito;
+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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyLong;
+import static org.powermock.api.mockito.PowerMockito.doAnswer;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+/**
+ * Unit tests for the {@link Kafka09Fetcher}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(KafkaConsumerThread.class)
+public class Kafka09FetcherTest {
+
+	@Test
+	public void testCommitDoesNotBlock() throws Exception {
+
+		// test data
+		final KafkaTopicPartition testPartition = new KafkaTopicPartition("test", 42);
+		final Map<KafkaTopicPartition, Long> testCommitData = new HashMap<>();
+		testCommitData.put(testPartition, 11L);
+
+		// to synchronize when the consumer is in its blocking method
+		final OneShotLatch sync = new OneShotLatch();
+
+		// ----- the mock consumer with blocking poll calls ----
+		final MultiShotLatch blockerLatch = new MultiShotLatch();
+		
+		KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
+		when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
+			
+			@Override
+			public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) throws InterruptedException {
+				sync.trigger();
+				blockerLatch.await();
+				return ConsumerRecords.empty();
+			}
+		});
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) {
+				blockerLatch.trigger();
+				return null;
+			}
+		}).when(mockConsumer).wakeup();
+
+		// make sure the fetcher creates the mock consumer
+		whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
+
+		// ----- create the test fetcher -----
+
+		@SuppressWarnings("unchecked")
+		SourceContext<String> sourceContext = mock(SourceContext.class);
+		List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition("test", 42));
+		KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
+
+		final Kafka09Fetcher<String> fetcher = new Kafka09Fetcher<>(
+				sourceContext,
+				topics,
+				null, /* periodic watermark extractor */
+				null, /* punctuated watermark extractor */
+				new TestProcessingTimeService(),
+				10, /* watermark interval */
+				this.getClass().getClassLoader(),
+				true, /* checkpointing */
+				"task_name",
+				new UnregisteredMetricsGroup(),
+				schema,
+				new Properties(),
+				0L,
+				false);
+
+		// ----- run the fetcher -----
+
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+		final Thread fetcherRunner = new Thread("fetcher runner") {
+
+			@Override
+			public void run() {
+				try {
+					fetcher.runFetchLoop();
+				} catch (Throwable t) {
+					error.set(t);
+				}
+			}
+		};
+		fetcherRunner.start();
+
+		// wait until the fetcher has reached the method of interest
+		sync.await();
+
+		// ----- trigger the offset commit -----
+		
+		final AtomicReference<Throwable> commitError = new AtomicReference<>();
+		final Thread committer = new Thread("committer runner") {
+			@Override
+			public void run() {
+				try {
+					fetcher.commitInternalOffsetsToKafka(testCommitData);
+				} catch (Throwable t) {
+					commitError.set(t);
+				}
+			}
+		};
+		committer.start();
+
+		// ----- ensure that the committer finishes in time  -----
+		committer.join(30000);
+		assertFalse("The committer did not finish in time", committer.isAlive());
+
+		// ----- test done, wait till the fetcher is done for a clean shutdown -----
+		fetcher.cancel();
+		fetcherRunner.join();
+
+		// check that there were no errors in the fetcher
+		final Throwable fetcherError = error.get();
+		if (fetcherError != null && !(fetcherError instanceof Handover.ClosedException)) {
+			throw new Exception("Exception in the fetcher", fetcherError);
+		}
+		final Throwable committerError = commitError.get();
+		if (committerError != null) {
+			throw new Exception("Exception in the committer", committerError);
+		}
+	}
+
+	@Test
+	public void ensureOffsetsGetCommitted() throws Exception {
+		
+		// test data
+		final KafkaTopicPartition testPartition1 = new KafkaTopicPartition("test", 42);
+		final KafkaTopicPartition testPartition2 = new KafkaTopicPartition("another", 99);
+		
+		final Map<KafkaTopicPartition, Long> testCommitData1 = new HashMap<>();
+		testCommitData1.put(testPartition1, 11L);
+		testCommitData1.put(testPartition2, 18L);
+
+		final Map<KafkaTopicPartition, Long> testCommitData2 = new HashMap<>();
+		testCommitData2.put(testPartition1, 19L);
+		testCommitData2.put(testPartition2, 28L);
+
+		final BlockingQueue<Map<TopicPartition, OffsetAndMetadata>> commitStore = new LinkedBlockingQueue<>();
+
+
+		// ----- the mock consumer with poll(), wakeup(), and commit(A)sync calls ----
+
+		final MultiShotLatch blockerLatch = new MultiShotLatch();
+
+		KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
+
+		when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
+			@Override
+			public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) throws InterruptedException {
+				blockerLatch.await();
+				return ConsumerRecords.empty();
+			}
+		});
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) {
+				blockerLatch.trigger();
+				return null;
+			}
+		}).when(mockConsumer).wakeup();
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) {
+				@SuppressWarnings("unchecked")
+				Map<TopicPartition, OffsetAndMetadata> offsets = 
+						(Map<TopicPartition, OffsetAndMetadata>) invocation.getArguments()[0];
+
+				OffsetCommitCallback callback = (OffsetCommitCallback) invocation.getArguments()[1];
+
+				commitStore.add(offsets);
+				callback.onComplete(offsets, null);
+
+				return null; 
+			}
+		}).when(mockConsumer).commitAsync(
+				Mockito.<Map<TopicPartition, OffsetAndMetadata>>any(), any(OffsetCommitCallback.class));
+
+		// make sure the fetcher creates the mock consumer
+		whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
+
+		// ----- create the test fetcher -----
+
+		@SuppressWarnings("unchecked")
+		SourceContext<String> sourceContext = mock(SourceContext.class);
+		List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition("test", 42));
+		KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
+
+		final Kafka09Fetcher<String> fetcher = new Kafka09Fetcher<>(
+				sourceContext,
+				topics,
+				null, /* periodic watermark extractor */
+				null, /* punctuated watermark extractor */
+				new TestProcessingTimeService(),
+				10, /* watermark interval */
+				this.getClass().getClassLoader(),
+				true, /* checkpointing */
+				"task_name",
+				new UnregisteredMetricsGroup(),
+				schema,
+				new Properties(),
+				0L,
+				false);
+
+
+		// ----- run the fetcher -----
+
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+		final Thread fetcherRunner = new Thread("fetcher runner") {
+
+			@Override
+			public void run() {
+				try {
+					fetcher.runFetchLoop();
+				} catch (Throwable t) {
+					error.set(t);
+				}
+			}
+		};
+		fetcherRunner.start();
+
+		// ----- trigger the first offset commit -----
+
+		fetcher.commitInternalOffsetsToKafka(testCommitData1);
+		Map<TopicPartition, OffsetAndMetadata> result1 = commitStore.take();
+
+		for (Entry<TopicPartition, OffsetAndMetadata> entry : result1.entrySet()) {
+			TopicPartition partition = entry.getKey();
+			if (partition.topic().equals("test")) {
+				assertEquals(42, partition.partition());
+				assertEquals(12L, entry.getValue().offset());
+			}
+			else if (partition.topic().equals("another")) {
+				assertEquals(99, partition.partition());
+				assertEquals(17L, entry.getValue().offset());
+			}
+		}
+
+		// ----- trigger the second offset commit -----
+
+		fetcher.commitInternalOffsetsToKafka(testCommitData2);
+		Map<TopicPartition, OffsetAndMetadata> result2 = commitStore.take();
+
+		for (Entry<TopicPartition, OffsetAndMetadata> entry : result2.entrySet()) {
+			TopicPartition partition = entry.getKey();
+			if (partition.topic().equals("test")) {
+				assertEquals(42, partition.partition());
+				assertEquals(20L, entry.getValue().offset());
+			}
+			else if (partition.topic().equals("another")) {
+				assertEquals(99, partition.partition());
+				assertEquals(27L, entry.getValue().offset());
+			}
+		}
+		
+		// ----- test done, wait till the fetcher is done for a clean shutdown -----
+		fetcher.cancel();
+		fetcherRunner.join();
+
+		// check that there were no errors in the fetcher
+		final Throwable caughtError = error.get();
+		if (caughtError != null && !(caughtError instanceof Handover.ClosedException)) {
+			throw new Exception("Exception in the fetcher", caughtError);
+		}
+	}
+
+	@Test
+	public void testCancellationWhenEmitBlocks() throws Exception {
+
+		// ----- some test data -----
+
+		final String topic = "test-topic";
+		final int partition = 3;
+		final byte[] payload = new byte[] {1, 2, 3, 4};
+
+		final List<ConsumerRecord<byte[], byte[]>> records = Arrays.asList(
+				new ConsumerRecord<byte[], byte[]>(topic, partition, 15, payload, payload),
+				new ConsumerRecord<byte[], byte[]>(topic, partition, 16, payload, payload),
+				new ConsumerRecord<byte[], byte[]>(topic, partition, 17, payload, payload));
+
+		final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> data = new HashMap<>();
+		data.put(new TopicPartition(topic, partition), records);
+
+		final ConsumerRecords<byte[], byte[]> consumerRecords = new ConsumerRecords<>(data);
+
+		// ----- the test consumer -----
+
+		final KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
+		when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
+			@Override
+			public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) {
+				return consumerRecords;
+			}
+		});
+
+		whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
+
+		// ----- build a fetcher -----
+
+		BlockingSourceContext<String> sourceContext = new BlockingSourceContext<>();
+		List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition(topic, partition));
+		KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
+
+		final Kafka09Fetcher<String> fetcher = new Kafka09Fetcher<>(
+				sourceContext,
+				topics,
+				null, /* periodic watermark extractor */
+				null, /* punctuated watermark extractor */
+				new TestProcessingTimeService(),
+				10, /* watermark interval */
+				this.getClass().getClassLoader(),
+				true, /* checkpointing */
+				"task_name",
+				new UnregisteredMetricsGroup(),
+				schema,
+				new Properties(),
+				0L,
+				false);
+
+
+		// ----- run the fetcher -----
+
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+		final Thread fetcherRunner = new Thread("fetcher runner") {
+
+			@Override
+			public void run() {
+				try {
+					fetcher.runFetchLoop();
+				} catch (Throwable t) {
+					error.set(t);
+				}
+			}
+		};
+		fetcherRunner.start();
+
+		// wait until the thread started to emit records to the source context
+		sourceContext.waitTillHasBlocker();
+
+		// now we try to cancel the fetcher, including the interruption usually done on the task thread
+		// once it has finished, there must be no more thread blocked on the source context
+		fetcher.cancel();
+		fetcherRunner.interrupt();
+		fetcherRunner.join();
+
+		assertFalse("fetcher threads did not properly finish", sourceContext.isStillBlocking());
+	}
+
+	// ------------------------------------------------------------------------
+	//  test utilities
+	// ------------------------------------------------------------------------
+
+	private static final class BlockingSourceContext<T> implements SourceContext<T> {
+
+		private final ReentrantLock lock = new ReentrantLock();
+		private final OneShotLatch inBlocking = new OneShotLatch();
+
+		@Override
+		public void collect(T element) {
+			block();
+		}
+
+		@Override
+		public void collectWithTimestamp(T element, long timestamp) {
+			block();
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+			block();
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return new Object();
+		}
+
+		@Override
+		public void close() {}
+
+		public void waitTillHasBlocker() throws InterruptedException {
+			inBlocking.await();
+		}
+
+		public boolean isStillBlocking() {
+			return lock.isLocked();
+		}
+
+		@SuppressWarnings({"InfiniteLoopStatement", "SynchronizationOnLocalVariableOrMethodParameter"})
+		private void block() {
+			lock.lock();
+			try {
+				inBlocking.trigger();
+
+				// put this thread to sleep indefinitely
+				final Object o = new Object();
+				while (true) {
+					synchronized (o) {
+						o.wait();
+					}
+				}
+			}
+			catch (InterruptedException e) {
+				// exit cleanly, simply reset the interruption flag
+				Thread.currentThread().interrupt();
+			}
+			finally {
+				lock.unlock();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
new file mode 100644
index 0000000..d18e2a9
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.junit.Test;
+
+public class Kafka09ITCase extends KafkaConsumerTestBase {
+
+	// ------------------------------------------------------------------------
+	//  Suite of Tests
+	// ------------------------------------------------------------------------
+
+	@Test(timeout = 60000)
+	public void testFailOnNoBroker() throws Exception {
+		runFailOnNoBrokerTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testConcurrentProducerConsumerTopology() throws Exception {
+		runSimpleConcurrentProducerConsumerTopology();
+	}
+
+
+	@Test(timeout = 60000)
+	public void testKeyValueSupport() throws Exception {
+		runKeyValueTest();
+	}
+
+	// --- canceling / failures ---
+
+	@Test(timeout = 60000)
+	public void testCancelingEmptyTopic() throws Exception {
+		runCancelingOnEmptyInputTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testCancelingFullTopic() throws Exception {
+		runCancelingOnFullInputTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testFailOnDeploy() throws Exception {
+		runFailOnDeployTest();
+	}
+
+
+	// --- source to partition mappings and exactly once ---
+
+	@Test(timeout = 60000)
+	public void testOneToOneSources() throws Exception {
+		runOneToOneExactlyOnceTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testOneSourceMultiplePartitions() throws Exception {
+		runOneSourceMultiplePartitionsExactlyOnceTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testMultipleSourcesOnePartition() throws Exception {
+		runMultipleSourcesOnePartitionExactlyOnceTest();
+	}
+
+	// --- broker failure ---
+
+	@Test(timeout = 60000)
+	public void testBrokerFailure() throws Exception {
+		runBrokerFailureTest();
+	}
+
+	// --- special executions ---
+
+	@Test(timeout = 60000)
+	public void testBigRecordJob() throws Exception {
+		runBigRecordTestTopology();
+	}
+
+	@Test(timeout = 60000)
+	public void testMultipleTopics() throws Exception {
+		runProduceConsumeMultipleTopics();
+	}
+
+	@Test(timeout = 60000)
+	public void testAllDeletes() throws Exception {
+		runAllDeletesTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testEndOfStream() throws Exception {
+		runEndOfStreamTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testMetrics() throws Throwable {
+		runMetricsTest();
+	}
+
+	// --- offset committing ---
+
+	@Test(timeout = 60000)
+	public void testCommitOffsetsToKafka() throws Exception {
+		runCommitOffsetsToKafka();
+	}
+
+	@Test(timeout = 60000)
+	public void testStartFromKafkaCommitOffsets() throws Exception {
+		runStartFromKafkaCommitOffsets();
+	}
+
+	@Test(timeout = 60000)
+	public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception {
+		runAutoOffsetRetrievalAndCommitToKafka();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
new file mode 100644
index 0000000..45f70ac
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import java.util.Properties;
+
+public class Kafka09JsonTableSinkTest extends KafkaTableSinkTestBase {
+
+	@Override
+	protected KafkaTableSink createTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner,
+			final FlinkKafkaProducerBase<Row> kafkaProducer) {
+
+		return new Kafka09JsonTableSink(topic, properties, partitioner) {
+			@Override
+			protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties,
+					SerializationSchema<Row> serializationSchema, KafkaPartitioner<Row> partitioner) {
+				return kafkaProducer;
+			}
+		};
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	protected SerializationSchema<Row> getSerializationSchema() {
+		return new JsonRowSerializationSchema(FIELD_NAMES);
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
new file mode 100644
index 0000000..4a75f50
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import java.util.Properties;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
+
+public class Kafka09JsonTableSourceTest extends KafkaTableSourceTestBase {
+
+	@Override
+	protected KafkaTableSource createTableSource(String topic, Properties properties, String[] fieldNames, TypeInformation<?>[] typeInfo) {
+		return new Kafka09JsonTableSource(topic, properties, fieldNames, typeInfo);
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	protected Class<DeserializationSchema<Row>> getDeserializationSchema() {
+		return (Class) JsonRowDeserializationSchema.class;
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	protected Class<FlinkKafkaConsumerBase<Row>> getFlinkKafkaConsumer() {
+		return (Class) FlinkKafkaConsumer09.class;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java
new file mode 100644
index 0000000..ae4f5b2
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+
+import org.junit.Test;
+
+@SuppressWarnings("serial")
+public class Kafka09ProducerITCase extends KafkaProducerTestBase {
+
+	@Test
+	public void testCustomPartitioning() {
+		runCustomPartitioningTest();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java
new file mode 100644
index 0000000..e748537
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.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;
+
+import org.apache.flink.test.util.SecureTestEnvironment;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/*
+ * Kafka Secure Connection (kerberos) IT test case
+ */
+public class Kafka09SecuredRunITCase extends KafkaConsumerTestBase {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(Kafka09SecuredRunITCase.class);
+
+	@BeforeClass
+	public static void prepare() throws IOException, ClassNotFoundException {
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    Starting Kafka09SecuredRunITCase ");
+		LOG.info("-------------------------------------------------------------------------");
+
+		SecureTestEnvironment.prepare(tempFolder);
+		SecureTestEnvironment.populateFlinkSecureConfigurations(getFlinkConfiguration());
+
+		startClusters(true);
+	}
+
+	@AfterClass
+	public static void shutDownServices() {
+		shutdownClusters();
+		SecureTestEnvironment.cleanup();
+	}
+
+
+	//timeout interval is large since in Travis, ZK connection timeout occurs frequently
+	//The timeout for the test case is 2 times timeout of ZK connection
+	@Test(timeout = 600000)
+	public void testMultipleTopics() throws Exception {
+		runProduceConsumeMultipleTopics();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
new file mode 100644
index 0000000..18b2aec
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.operators.StreamSink;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
+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.Collections;
+import java.util.concurrent.Future;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(FlinkKafkaProducerBase.class)
+public class KafkaProducerTest extends TestLogger {
+	
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testPropagateExceptions() {
+		try {
+			// mock kafka producer
+			KafkaProducer<?, ?> kafkaProducerMock = mock(KafkaProducer.class);
+			
+			// partition setup
+			when(kafkaProducerMock.partitionsFor(anyString())).thenReturn(
+				// returning a unmodifiable list to mimic KafkaProducer#partitionsFor() behaviour
+				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)))
+				.thenAnswer(new Answer<Future<RecordMetadata>>() {
+					@Override
+					public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
+						Callback callback = (Callback) invocation.getArguments()[1];
+						callback.onCompletion(null, new Exception("Test error"));
+						return null;
+					}
+				});
+			
+			// make sure the FlinkKafkaProducer instantiates our mock producer
+			whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock);
+			
+			// (1) producer that propagates errors
+
+			FlinkKafkaProducer09<String> producerPropagating = new FlinkKafkaProducer09<>(
+					"mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), null);
+
+			OneInputStreamOperatorTestHarness<String, Object> testHarness =
+					new OneInputStreamOperatorTestHarness<>(new StreamSink(producerPropagating));
+
+			testHarness.open();
+
+			try {
+				testHarness.processElement(new StreamRecord<>("value"));
+				testHarness.processElement(new StreamRecord<>("value"));
+				fail("This should fail with an exception");
+			}
+			catch (Exception e) {
+				assertNotNull(e.getCause());
+				assertNotNull(e.getCause().getMessage());
+				assertTrue(e.getCause().getMessage().contains("Test error"));
+			}
+
+			// (2) producer that only logs errors
+
+			FlinkKafkaProducer09<String> producerLogging = new FlinkKafkaProducer09<>(
+					"mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), null);
+			producerLogging.setLogFailuresOnly(true);
+
+			testHarness = new OneInputStreamOperatorTestHarness<>(new StreamSink(producerLogging));
+
+			testHarness.open();
+
+			testHarness.processElement(new StreamRecord<>("value"));
+			testHarness.processElement(new StreamRecord<>("value"));
+
+			testHarness.close();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
new file mode 100644
index 0000000..1802e0c
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.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;
+
+import kafka.admin.AdminUtils;
+import kafka.common.KafkaException;
+import kafka.api.PartitionMetadata;
+import kafka.network.SocketServer;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+import kafka.utils.SystemTime$;
+import kafka.utils.ZkUtils;
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.util.NetUtils;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.Seq;
+
+import java.io.File;
+import java.net.BindException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * An implementation of the KafkaServerProvider for Kafka 0.9
+ */
+public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class);
+	private File tmpZkDir;
+	private File tmpKafkaParent;
+	private List<File> tmpKafkaDirs;
+	private List<KafkaServer> brokers;
+	private TestingServer zookeeper;
+	private String zookeeperConnectionString;
+	private String brokerConnectionString = "";
+	private Properties standardProps;
+	private Properties additionalServerProperties;
+	private boolean secureMode = false;
+	// 6 seconds is default. Seems to be too small for travis. 30 seconds
+	private String zkTimeout = "30000";
+
+	public String getBrokerConnectionString() {
+		return brokerConnectionString;
+	}
+
+	@Override
+	public Properties getStandardProperties() {
+		return standardProps;
+	}
+
+	@Override
+	public String getVersion() {
+		return "0.9";
+	}
+
+	@Override
+	public List<KafkaServer> getBrokers() {
+		return brokers;
+	}
+
+	@Override
+	public <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, KeyedDeserializationSchema<T> readSchema, Properties props) {
+		return new FlinkKafkaConsumer09<>(topics, readSchema, props);
+	}
+
+	@Override
+	public <T> StreamSink<T> getProducerSink(
+			String topic,
+			KeyedSerializationSchema<T> serSchema,
+			Properties props,
+			KafkaPartitioner<T> partitioner) {
+		FlinkKafkaProducer09<T> prod = new FlinkKafkaProducer09<>(topic, serSchema, props, partitioner);
+		prod.setFlushOnCheckpoint(true);
+		return new StreamSink<>(prod);
+	}
+
+	@Override
+	public <T> DataStreamSink<T> produceIntoKafka(DataStream<T> stream, String topic, KeyedSerializationSchema<T> serSchema, Properties props, KafkaPartitioner<T> partitioner) {
+		FlinkKafkaProducer09<T> prod = new FlinkKafkaProducer09<>(topic, serSchema, props, partitioner);
+		prod.setFlushOnCheckpoint(true);
+		return stream.addSink(prod);
+	}
+
+	@Override
+	public KafkaOffsetHandler createOffsetHandler(Properties props) {
+		return new KafkaOffsetHandlerImpl(props);
+	}
+
+	@Override
+	public void restartBroker(int leaderId) throws Exception {
+		brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId)));
+	}
+
+	@Override
+	public int getLeaderToShutDown(String topic) throws Exception {
+		ZkUtils zkUtils = getZkUtils();
+		try {
+			PartitionMetadata firstPart = null;
+			do {
+				if (firstPart != null) {
+					LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
+					// not the first try. Sleep a bit
+					Thread.sleep(150);
+				}
+
+				Seq<PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkUtils).partitionsMetadata();
+				firstPart = partitionMetadata.head();
+			}
+			while (firstPart.errorCode() != 0);
+
+			return firstPart.leader().get().id();
+		} finally {
+			zkUtils.close();
+		}
+	}
+
+	@Override
+	public int getBrokerId(KafkaServer server) {
+		return server.config().brokerId();
+	}
+
+	@Override
+	public boolean isSecureRunSupported() {
+		return true;
+	}
+
+	@Override
+	public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) {
+
+		//increase the timeout since in Travis ZK connection takes long time for secure connection.
+		if(secureMode) {
+			//run only one kafka server to avoid multiple ZK connections from many instances - Travis timeout
+			numKafkaServers = 1;
+			zkTimeout = String.valueOf(Integer.parseInt(zkTimeout) * 15);
+		}
+
+		this.additionalServerProperties = additionalServerProperties;
+		this.secureMode = secureMode;
+		File tempDir = new File(System.getProperty("java.io.tmpdir"));
+
+		tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
+		assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs());
+
+		tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString()));
+		assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs());
+
+		tmpKafkaDirs = new ArrayList<>(numKafkaServers);
+		for (int i = 0; i < numKafkaServers; i++) {
+			File tmpDir = new File(tmpKafkaParent, "server-" + i);
+			assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
+			tmpKafkaDirs.add(tmpDir);
+		}
+
+		zookeeper = null;
+		brokers = null;
+
+		try {
+			LOG.info("Starting Zookeeper");
+			zookeeper = new TestingServer(-1, tmpZkDir);
+			zookeeperConnectionString = zookeeper.getConnectString();
+			LOG.info("zookeeperConnectionString: {}", zookeeperConnectionString);
+
+			LOG.info("Starting KafkaServer");
+			brokers = new ArrayList<>(numKafkaServers);
+
+			for (int i = 0; i < numKafkaServers; i++) {
+				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i)));
+
+				SocketServer socketServer = brokers.get(i).socketServer();
+				if(secureMode) {
+					brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.SASL_PLAINTEXT)) + ",";
+				} else {
+					brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ",";
+				}
+			}
+
+			LOG.info("ZK and KafkaServer started.");
+		}
+		catch (Throwable t) {
+			t.printStackTrace();
+			fail("Test setup failed: " + t.getMessage());
+		}
+
+		LOG.info("brokerConnectionString --> {}", brokerConnectionString);
+
+		standardProps = new Properties();
+		standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
+		standardProps.setProperty("bootstrap.servers", brokerConnectionString);
+		standardProps.setProperty("group.id", "flink-tests");
+		standardProps.setProperty("enable.auto.commit", "false");
+		standardProps.setProperty("zookeeper.session.timeout.ms", zkTimeout);
+		standardProps.setProperty("zookeeper.connection.timeout.ms", zkTimeout);
+		standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning. (earliest is kafka 0.9 value)
+		standardProps.setProperty("max.partition.fetch.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
+
+	}
+
+	@Override
+	public void shutdown() {
+		for (KafkaServer broker : brokers) {
+			if (broker != null) {
+				broker.shutdown();
+			}
+		}
+		brokers.clear();
+
+		if (zookeeper != null) {
+			try {
+				zookeeper.stop();
+				zookeeper.close();
+			}
+			catch (Exception e) {
+				LOG.warn("ZK.stop() failed", e);
+			}
+			zookeeper = null;
+		}
+
+		// clean up the temp spaces
+
+		if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpKafkaParent);
+			}
+			catch (Exception e) {
+				// ignore
+			}
+		}
+		if (tmpZkDir != null && tmpZkDir.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpZkDir);
+			}
+			catch (Exception e) {
+				// ignore
+			}
+		}
+	}
+
+	public ZkUtils getZkUtils() {
+		LOG.info("In getZKUtils:: zookeeperConnectionString = {}", zookeeperConnectionString);
+		ZkClient creator = new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
+				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
+		return ZkUtils.apply(creator, false);
+	}
+
+	@Override
+	public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor, Properties topicConfig) {
+		// create topic with one client
+		LOG.info("Creating topic {}", topic);
+
+		ZkUtils zkUtils = getZkUtils();
+		try {
+			AdminUtils.createTopic(zkUtils, topic, numberOfPartitions, replicationFactor, topicConfig);
+		} finally {
+			zkUtils.close();
+		}
+
+		LOG.info("Topic {} create request is successfully posted", topic);
+
+		// validate that the topic has been created
+		final long deadline = System.currentTimeMillis() + Integer.parseInt(zkTimeout);
+		do {
+			try {
+				if(secureMode) {
+					//increase wait time since in Travis ZK timeout occurs frequently
+					int wait = Integer.parseInt(zkTimeout) / 100;
+					LOG.info("waiting for {} msecs before the topic {} can be checked", wait, topic);
+					Thread.sleep(wait);
+				} else {
+					Thread.sleep(100);
+				}
+
+			} catch (InterruptedException e) {
+				// restore interrupted state
+			}
+			// we could use AdminUtils.topicExists(zkUtils, topic) here, but it's results are
+			// not always correct.
+
+			LOG.info("Validating if the topic {} has been created or not", topic);
+
+			// create a new ZK utils connection
+			ZkUtils checkZKConn = getZkUtils();
+			if(AdminUtils.topicExists(checkZKConn, topic)) {
+				LOG.info("topic {} has been created successfully", topic);
+				checkZKConn.close();
+				return;
+			}
+			LOG.info("topic {} has not been created yet. Will check again...", topic);
+			checkZKConn.close();
+		}
+		while (System.currentTimeMillis() < deadline);
+		fail("Test topic could not be created");
+	}
+
+	@Override
+	public void deleteTestTopic(String topic) {
+		ZkUtils zkUtils = getZkUtils();
+		try {
+			LOG.info("Deleting topic {}", topic);
+
+			ZkClient zk = new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
+				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
+
+			AdminUtils.deleteTopic(zkUtils, topic);
+
+			zk.close();
+		} finally {
+			zkUtils.close();
+		}
+	}
+
+	/**
+	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
+	 */
+	protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception {
+		Properties kafkaProperties = new Properties();
+
+		// properties have to be Strings
+		kafkaProperties.put("advertised.host.name", KAFKA_HOST);
+		kafkaProperties.put("broker.id", Integer.toString(brokerId));
+		kafkaProperties.put("log.dir", tmpFolder.toString());
+		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
+		kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
+		kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
+
+		// for CI stability, increase zookeeper session timeout
+		kafkaProperties.put("zookeeper.session.timeout.ms", zkTimeout);
+		kafkaProperties.put("zookeeper.connection.timeout.ms", zkTimeout);
+		if(additionalServerProperties != null) {
+			kafkaProperties.putAll(additionalServerProperties);
+		}
+
+		final int numTries = 5;
+
+		for (int i = 1; i <= numTries; i++) {
+			int kafkaPort = NetUtils.getAvailablePort();
+			kafkaProperties.put("port", Integer.toString(kafkaPort));
+
+			//to support secure kafka cluster
+			if(secureMode) {
+				LOG.info("Adding Kafka secure configurations");
+				kafkaProperties.put("listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort);
+				kafkaProperties.put("advertised.listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort);
+				kafkaProperties.putAll(getSecureProperties());
+			}
+
+			KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
+
+			try {
+				scala.Option<String> stringNone = scala.Option.apply(null);
+				KafkaServer server = new KafkaServer(kafkaConfig, SystemTime$.MODULE$, stringNone);
+				server.startup();
+				return server;
+			}
+			catch (KafkaException e) {
+				if (e.getCause() instanceof BindException) {
+					// port conflict, retry...
+					LOG.info("Port conflict when starting Kafka Broker. Retrying...");
+				}
+				else {
+					throw e;
+				}
+			}
+		}
+
+		throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
+	}
+
+	public Properties getSecureProperties() {
+		Properties prop = new Properties();
+		if(secureMode) {
+			prop.put("security.inter.broker.protocol", "SASL_PLAINTEXT");
+			prop.put("security.protocol", "SASL_PLAINTEXT");
+			prop.put("sasl.kerberos.service.name", "kafka");
+
+			//add special timeout for Travis
+			prop.setProperty("zookeeper.session.timeout.ms", zkTimeout);
+			prop.setProperty("zookeeper.connection.timeout.ms", zkTimeout);
+			prop.setProperty("metadata.fetch.timeout.ms","120000");
+		}
+		return prop;
+	}
+
+	private class KafkaOffsetHandlerImpl implements KafkaOffsetHandler {
+
+		private final KafkaConsumer<byte[], byte[]> offsetClient;
+
+		public KafkaOffsetHandlerImpl(Properties props) {
+			offsetClient = new KafkaConsumer<>(props);
+		}
+
+		@Override
+		public Long getCommittedOffset(String topicName, int partition) {
+			OffsetAndMetadata committed = offsetClient.committed(new TopicPartition(topicName, partition));
+			return (committed != null) ? committed.offset() : null;
+		}
+
+		@Override
+		public void close() {
+			offsetClient.close();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java
new file mode 100644
index 0000000..25040eb
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java
@@ -0,0 +1,387 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.streaming.connectors.kafka.internal.Handover.WakeupException;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for the {@link Handover} between Kafka Consumer Thread and the fetcher's main thread. 
+ */
+public class HandoverTest {
+
+	// ------------------------------------------------------------------------
+	//  test produce / consumer
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testWithVariableProducer() throws Exception {
+		runProducerConsumerTest(500, 2, 0);
+	}
+
+	@Test
+	public void testWithVariableConsumer() throws Exception {
+		runProducerConsumerTest(500, 0, 2);
+	}
+
+	@Test
+	public void testWithVariableBoth() throws Exception {
+		runProducerConsumerTest(500, 2, 2);
+	}
+
+	// ------------------------------------------------------------------------
+	//  test error propagation
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testPublishErrorOnEmptyHandover() throws Exception {
+		final Handover handover = new Handover();
+
+		Exception error = new Exception();
+		handover.reportError(error);
+
+		try {
+			handover.pollNext();
+			fail("should throw an exception");
+		}
+		catch (Exception e) {
+			assertEquals(error, e);
+		}
+	}
+
+	@Test
+	public void testPublishErrorOnFullHandover() throws Exception {
+		final Handover handover = new Handover();
+		handover.produce(createTestRecords());
+
+		IOException error = new IOException();
+		handover.reportError(error);
+
+		try {
+			handover.pollNext();
+			fail("should throw an exception");
+		}
+		catch (Exception e) {
+			assertEquals(error, e);
+		}
+	}
+
+	@Test
+	public void testExceptionMarksClosedOnEmpty() throws Exception {
+		final Handover handover = new Handover();
+
+		IllegalStateException error = new IllegalStateException();
+		handover.reportError(error);
+
+		try {
+			handover.produce(createTestRecords());
+			fail("should throw an exception");
+		}
+		catch (Handover.ClosedException e) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testExceptionMarksClosedOnFull() throws Exception {
+		final Handover handover = new Handover();
+		handover.produce(createTestRecords());
+
+		LinkageError error = new LinkageError();
+		handover.reportError(error);
+
+		try {
+			handover.produce(createTestRecords());
+			fail("should throw an exception");
+		}
+		catch (Handover.ClosedException e) {
+			// expected
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  test closing behavior
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testCloseEmptyForConsumer() throws Exception {
+		final Handover handover = new Handover();
+		handover.close();
+
+		try {
+			handover.pollNext();
+			fail("should throw an exception");
+		}
+		catch (Handover.ClosedException e) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testCloseFullForConsumer() throws Exception {
+		final Handover handover = new Handover();
+		handover.produce(createTestRecords());
+		handover.close();
+
+		try {
+			handover.pollNext();
+			fail("should throw an exception");
+		}
+		catch (Handover.ClosedException e) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testCloseEmptyForProducer() throws Exception {
+		final Handover handover = new Handover();
+		handover.close();
+
+		try {
+			handover.produce(createTestRecords());
+			fail("should throw an exception");
+		}
+		catch (Handover.ClosedException e) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testCloseFullForProducer() throws Exception {
+		final Handover handover = new Handover();
+		handover.produce(createTestRecords());
+		handover.close();
+
+		try {
+			handover.produce(createTestRecords());
+			fail("should throw an exception");
+		}
+		catch (Handover.ClosedException e) {
+			// expected
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  test wake up behavior
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testWakeupDoesNotWakeWhenEmpty() throws Exception {
+		Handover handover = new Handover();
+		handover.wakeupProducer();
+
+		// produce into a woken but empty handover
+		try {
+			handover.produce(createTestRecords());
+		}
+		catch (Handover.WakeupException e) {
+			fail();
+		}
+
+		// handover now has records, next time we wakeup and produce it needs
+		// to throw an exception
+		handover.wakeupProducer();
+		try {
+			handover.produce(createTestRecords());
+			fail("should throw an exception");
+		}
+		catch (Handover.WakeupException e) {
+			// expected
+		}
+
+		// empty the handover
+		assertNotNull(handover.pollNext());
+		
+		// producing into an empty handover should work
+		try {
+			handover.produce(createTestRecords());
+		}
+		catch (Handover.WakeupException e) {
+			fail();
+		}
+	}
+
+	@Test
+	public void testWakeupWakesOnlyOnce() throws Exception {
+		// create a full handover
+		final Handover handover = new Handover();
+		handover.produce(createTestRecords());
+
+		handover.wakeupProducer();
+
+		try {
+			handover.produce(createTestRecords());
+			fail();
+		} catch (WakeupException e) {
+			// expected
+		}
+
+		CheckedThread producer = new CheckedThread() {
+			@Override
+			public void go() throws Exception {
+				handover.produce(createTestRecords());
+			}
+		};
+		producer.start();
+
+		// the producer must go blocking
+		producer.waitUntilThreadHoldsLock(10000);
+
+		// release the thread by consuming something
+		assertNotNull(handover.pollNext());
+		producer.sync();
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private void runProducerConsumerTest(int numRecords, int maxProducerDelay, int maxConsumerDelay) throws Exception {
+		// generate test data
+		@SuppressWarnings({"unchecked", "rawtypes"})
+		final ConsumerRecords<byte[], byte[]>[] data = new ConsumerRecords[numRecords];
+		for (int i = 0; i < numRecords; i++) {
+			data[i] = createTestRecords();
+		}
+
+		final Handover handover = new Handover();
+
+		ProducerThread producer = new ProducerThread(handover, data, maxProducerDelay);
+		ConsumerThread consumer = new ConsumerThread(handover, data, maxConsumerDelay);
+
+		consumer.start();
+		producer.start();
+
+		// sync first on the consumer, so it propagates assertion errors
+		consumer.sync();
+		producer.sync();
+	}
+
+	@SuppressWarnings("unchecked")
+	private static ConsumerRecords<byte[], byte[]> createTestRecords() {
+		return mock(ConsumerRecords.class);
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static abstract class CheckedThread extends Thread {
+
+		private volatile Throwable error;
+
+		public abstract void go() throws Exception;
+
+		@Override
+		public void run() {
+			try {
+				go();
+			}
+			catch (Throwable t) {
+				error = t;
+			}
+		}
+
+		public void sync() throws Exception {
+			join();
+			if (error != null) {
+				ExceptionUtils.rethrowException(error, error.getMessage());
+			}
+		}
+
+		public void waitUntilThreadHoldsLock(long timeoutMillis) throws InterruptedException, TimeoutException {
+			final long deadline = System.nanoTime() + timeoutMillis * 1_000_000;
+			
+			while (!isBlockedOrWaiting() && (System.nanoTime() < deadline)) {
+				Thread.sleep(1);
+			}
+
+			if (!isBlockedOrWaiting()) {
+				throw new TimeoutException();
+			}
+		}
+
+		private boolean isBlockedOrWaiting() {
+			State state = getState();
+			return state == State.BLOCKED || state == State.WAITING || state == State.TIMED_WAITING;
+		}
+	}
+
+	private static class ProducerThread extends CheckedThread {
+
+		private final Random rnd = new Random();
+		private final Handover handover;
+		private final ConsumerRecords<byte[], byte[]>[] data;
+		private final int maxDelay;
+
+		private ProducerThread(Handover handover, ConsumerRecords<byte[], byte[]>[] data, int maxDelay) {
+			this.handover = handover;
+			this.data = data;
+			this.maxDelay = maxDelay;
+		}
+
+		@Override
+		public void go() throws Exception {
+			for (ConsumerRecords<byte[], byte[]> rec : data) {
+				handover.produce(rec);
+
+				if (maxDelay > 0) {
+					int delay = rnd.nextInt(maxDelay);
+					Thread.sleep(delay);
+				}
+			}
+		}
+	}
+
+	private static class ConsumerThread extends CheckedThread {
+
+		private final Random rnd = new Random();
+		private final Handover handover;
+		private final ConsumerRecords<byte[], byte[]>[] data;
+		private final int maxDelay;
+
+		private ConsumerThread(Handover handover, ConsumerRecords<byte[], byte[]>[] data, int maxDelay) {
+			this.handover = handover;
+			this.data = data;
+			this.maxDelay = maxDelay;
+		}
+
+		@Override
+		public void go() throws Exception {
+			for (ConsumerRecords<byte[], byte[]> rec : data) {
+				ConsumerRecords<byte[], byte[]> next = handover.pollNext();
+
+				assertEquals(rec, next);
+
+				if (maxDelay > 0) {
+					int delay = rnd.nextInt(maxDelay);
+					Thread.sleep(delay);
+				}
+			}
+		}
+	}
+}

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

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


[51/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
[FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

This closes #2897.


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

Branch: refs/heads/master
Commit: de4fe3b7392948807753d65d13f3da968e6c7de0
Parents: cc006ff
Author: Fabian Hueske <fh...@apache.org>
Authored: Tue Nov 29 13:57:30 2016 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Dec 2 14:28:35 2016 +0100

----------------------------------------------------------------------
 .gitignore                                      |    2 +-
 flink-batch-connectors/flink-avro/pom.xml       |  216 --
 .../apache/flink/api/avro/DataInputDecoder.java |  213 --
 .../flink/api/avro/DataOutputEncoder.java       |  183 --
 .../api/avro/FSDataInputStreamWrapper.java      |   68 -
 .../flink/api/java/io/AvroInputFormat.java      |  207 --
 .../flink/api/java/io/AvroOutputFormat.java     |  189 --
 .../src/test/assembly/test-assembly.xml         |   36 -
 .../api/avro/AvroExternalJarProgramITCase.java  |   80 -
 .../flink/api/avro/AvroOutputFormatITCase.java  |  176 --
 .../flink/api/avro/EncoderDecoderTest.java      |  528 -----
 .../avro/testjar/AvroExternalJarProgram.java    |  219 --
 .../apache/flink/api/io/avro/AvroPojoTest.java  |  255 ---
 .../api/io/avro/AvroRecordInputFormatTest.java  |  458 ----
 .../io/avro/AvroSplittableInputFormatTest.java  |  326 ---
 .../api/io/avro/example/AvroTypeExample.java    |  108 -
 .../apache/flink/api/io/avro/example/User.java  |  269 ---
 .../io/AvroInputFormatTypeExtractionTest.java   |   81 -
 .../flink/api/java/io/AvroOutputFormatTest.java |  154 --
 .../src/test/resources/avro/user.avsc           |   35 -
 .../src/test/resources/log4j-test.properties    |   27 -
 .../src/test/resources/logback-test.xml         |   29 -
 .../flink-avro/src/test/resources/testdata.avro |  Bin 4572 -> 0 bytes
 .../flink-hadoop-compatibility/pom.xml          |  182 --
 .../api/java/typeutils/WritableTypeInfo.java    |  154 --
 .../typeutils/runtime/WritableComparator.java   |  188 --
 .../typeutils/runtime/WritableSerializer.java   |  152 --
 .../flink/hadoopcompatibility/HadoopInputs.java |  118 --
 .../flink/hadoopcompatibility/HadoopUtils.java  |   52 -
 .../mapred/HadoopMapFunction.java               |  133 --
 .../mapred/HadoopReduceCombineFunction.java     |  168 --
 .../mapred/HadoopReduceFunction.java            |  142 --
 .../mapred/wrapper/HadoopOutputCollector.java   |   59 -
 .../wrapper/HadoopTupleUnwrappingIterator.java  |   94 -
 .../scala/HadoopInputs.scala                    |  143 --
 .../java/typeutils/WritableExtractionTest.java  |  206 --
 .../java/typeutils/WritableInfoParserTest.java  |   84 -
 .../java/typeutils/WritableTypeInfoTest.java    |   72 -
 .../typeutils/runtime/StringArrayWritable.java  |   83 -
 .../runtime/WritableComparatorTest.java         |   53 -
 .../runtime/WritableComparatorUUIDTest.java     |   46 -
 .../api/java/typeutils/runtime/WritableID.java  |   78 -
 .../runtime/WritableSerializerTest.java         |   50 -
 .../runtime/WritableSerializerUUIDTest.java     |   50 -
 .../hadoopcompatibility/HadoopUtilsTest.java    |   34 -
 .../mapred/HadoopMapFunctionITCase.java         |  182 --
 .../mapred/HadoopMapredITCase.java              |   47 -
 .../HadoopReduceCombineFunctionITCase.java      |  265 ---
 .../mapred/HadoopReduceFunctionITCase.java      |  213 --
 .../mapred/HadoopTestData.java                  |   62 -
 .../example/HadoopMapredCompatWordCount.java    |  133 --
 .../HadoopTupleUnwrappingIteratorTest.java      |  139 --
 .../mapreduce/HadoopInputOutputITCase.java      |   47 -
 .../mapreduce/example/WordCount.java            |  119 --
 .../src/test/resources/log4j-test.properties    |   27 -
 .../src/test/resources/logback-test.xml         |   29 -
 flink-batch-connectors/flink-hbase/pom.xml      |  264 ---
 .../flink/addons/hbase/TableInputFormat.java    |  289 ---
 .../flink/addons/hbase/TableInputSplit.java     |   89 -
 .../hbase/HBaseTestingClusterAutostarter.java   |  238 ---
 .../addons/hbase/TableInputFormatITCase.java    |  120 --
 .../hbase/example/HBaseFlinkTestConstants.java  |   28 -
 .../addons/hbase/example/HBaseReadExample.java  |   92 -
 .../addons/hbase/example/HBaseWriteExample.java |  202 --
 .../hbase/example/HBaseWriteStreamExample.java  |  113 -
 .../src/test/resources/log4j-test.properties    |   23 -
 flink-batch-connectors/flink-hcatalog/pom.xml   |  182 --
 .../flink/hcatalog/HCatInputFormatBase.java     |  410 ----
 .../flink/hcatalog/java/HCatInputFormat.java    |  160 --
 .../flink/hcatalog/scala/HCatInputFormat.scala  |  229 --
 flink-batch-connectors/flink-jdbc/pom.xml       |   66 -
 .../flink/api/java/io/jdbc/JDBCInputFormat.java |  404 ----
 .../api/java/io/jdbc/JDBCOutputFormat.java      |  315 ---
 .../split/GenericParameterValuesProvider.java   |   44 -
 .../split/NumericBetweenParametersProvider.java |   66 -
 .../io/jdbc/split/ParameterValuesProvider.java  |   35 -
 .../flink/api/java/io/jdbc/JDBCFullTest.java    |  101 -
 .../api/java/io/jdbc/JDBCInputFormatTest.java   |  247 ---
 .../api/java/io/jdbc/JDBCOutputFormatTest.java  |  169 --
 .../flink/api/java/io/jdbc/JDBCTestBase.java    |  183 --
 .../src/test/resources/log4j-test.properties    |   19 -
 .../src/test/resources/logback-test.xml         |   29 -
 flink-batch-connectors/pom.xml                  |   45 -
 flink-connectors/flink-avro/pom.xml             |  216 ++
 .../apache/flink/api/avro/DataInputDecoder.java |  213 ++
 .../flink/api/avro/DataOutputEncoder.java       |  183 ++
 .../api/avro/FSDataInputStreamWrapper.java      |   68 +
 .../flink/api/java/io/AvroInputFormat.java      |  207 ++
 .../flink/api/java/io/AvroOutputFormat.java     |  189 ++
 .../src/test/assembly/test-assembly.xml         |   36 +
 .../api/avro/AvroExternalJarProgramITCase.java  |   80 +
 .../flink/api/avro/AvroOutputFormatITCase.java  |  176 ++
 .../flink/api/avro/EncoderDecoderTest.java      |  528 +++++
 .../avro/testjar/AvroExternalJarProgram.java    |  219 ++
 .../apache/flink/api/io/avro/AvroPojoTest.java  |  255 +++
 .../api/io/avro/AvroRecordInputFormatTest.java  |  458 ++++
 .../io/avro/AvroSplittableInputFormatTest.java  |  326 +++
 .../api/io/avro/example/AvroTypeExample.java    |  108 +
 .../apache/flink/api/io/avro/example/User.java  |  269 +++
 .../io/AvroInputFormatTypeExtractionTest.java   |   81 +
 .../flink/api/java/io/AvroOutputFormatTest.java |  154 ++
 .../src/test/resources/avro/user.avsc           |   35 +
 .../src/test/resources/log4j-test.properties    |   27 +
 .../src/test/resources/logback-test.xml         |   29 +
 .../flink-avro/src/test/resources/testdata.avro |  Bin 0 -> 4572 bytes
 .../flink-connector-cassandra/pom.xml           |  179 ++
 .../cassandra/CassandraInputFormat.java         |  131 ++
 .../cassandra/CassandraOutputFormat.java        |  125 ++
 .../cassandra/CassandraCommitter.java           |  151 ++
 .../connectors/cassandra/CassandraPojoSink.java |   67 +
 .../connectors/cassandra/CassandraSink.java     |  329 +++
 .../connectors/cassandra/CassandraSinkBase.java |   98 +
 .../cassandra/CassandraTupleSink.java           |   59 +
 .../cassandra/CassandraTupleWriteAheadSink.java |  159 ++
 .../connectors/cassandra/ClusterBuilder.java    |   43 +
 .../cassandra/example/BatchExample.java         |   77 +
 .../cassandra/CassandraConnectorITCase.java     |  440 ++++
 .../CassandraTupleWriteAheadSinkTest.java       |  127 ++
 .../streaming/connectors/cassandra/Pojo.java    |   65 +
 .../example/CassandraPojoSinkExample.java       |   62 +
 .../example/CassandraTupleSinkExample.java      |   62 +
 .../CassandraTupleWriteAheadSinkExample.java    |   96 +
 .../connectors/cassandra/example/Message.java   |   56 +
 .../src/test/resources/cassandra.yaml           |   43 +
 .../src/test/resources/log4j-test.properties    |   29 +
 .../flink-connector-elasticsearch/pom.xml       |   90 +
 .../elasticsearch/ElasticsearchSink.java        |  315 +++
 .../elasticsearch/IndexRequestBuilder.java      |   66 +
 .../elasticsearch/ElasticsearchSinkITCase.java  |  205 ++
 .../examples/ElasticsearchExample.java          |   80 +
 .../src/test/resources/log4j-test.properties    |   27 +
 .../src/test/resources/logback-test.xml         |   30 +
 .../flink-connector-elasticsearch2/pom.xml      |   83 +
 .../elasticsearch2/BulkProcessorIndexer.java    |   35 +
 .../elasticsearch2/ElasticsearchSink.java       |  257 +++
 .../ElasticsearchSinkFunction.java              |   60 +
 .../elasticsearch2/RequestIndexer.java          |   25 +
 .../elasticsearch2/ElasticsearchSinkITCase.java |  233 ++
 .../examples/ElasticsearchExample.java          |   90 +
 .../src/test/resources/log4j-test.properties    |   27 +
 .../src/test/resources/logback-test.xml         |   30 +
 .../flink-connector-filesystem/pom.xml          |  163 ++
 .../connectors/fs/AvroKeyValueSinkWriter.java   |  309 +++
 .../flink/streaming/connectors/fs/Bucketer.java |   55 +
 .../flink/streaming/connectors/fs/Clock.java    |   33 +
 .../connectors/fs/DateTimeBucketer.java         |  126 ++
 .../connectors/fs/NonRollingBucketer.java       |   47 +
 .../streaming/connectors/fs/RollingSink.java    |  916 ++++++++
 .../connectors/fs/SequenceFileWriter.java       |  151 ++
 .../connectors/fs/StreamWriterBase.java         |  152 ++
 .../streaming/connectors/fs/StringWriter.java   |   86 +
 .../streaming/connectors/fs/SystemClock.java    |   29 +
 .../flink/streaming/connectors/fs/Writer.java   |   73 +
 .../fs/bucketing/BasePathBucketer.java          |   39 +
 .../connectors/fs/bucketing/Bucketer.java       |   47 +
 .../connectors/fs/bucketing/BucketingSink.java  | 1082 ++++++++++
 .../fs/bucketing/DateTimeBucketer.java          |  102 +
 .../src/main/resources/log4j.properties         |   27 +
 .../fs/RollingSinkFaultToleranceITCase.java     |  300 +++
 .../connectors/fs/RollingSinkITCase.java        |  991 +++++++++
 .../connectors/fs/RollingSinkSecuredITCase.java |  252 +++
 .../BucketingSinkFaultToleranceITCase.java      |  297 +++
 .../fs/bucketing/BucketingSinkTest.java         |  867 ++++++++
 .../src/test/resources/log4j-test.properties    |   29 +
 .../src/test/resources/logback-test.xml         |   30 +
 flink-connectors/flink-connector-flume/pom.xml  |  175 ++
 .../streaming/connectors/flume/FlumeSink.java   |  141 ++
 .../flink-connector-kafka-0.10/pom.xml          |  205 ++
 .../connectors/kafka/FlinkKafkaConsumer010.java |  153 ++
 .../connectors/kafka/FlinkKafkaProducer010.java |  398 ++++
 .../kafka/Kafka010JsonTableSource.java          |   71 +
 .../connectors/kafka/Kafka010TableSource.java   |   75 +
 .../kafka/internal/Kafka010Fetcher.java         |  104 +
 .../internal/KafkaConsumerCallBridge010.java    |   40 +
 .../src/main/resources/log4j.properties         |   29 +
 .../connectors/kafka/Kafka010FetcherTest.java   |  484 +++++
 .../connectors/kafka/Kafka010ITCase.java        |  313 +++
 .../kafka/Kafka010ProducerITCase.java           |   33 +
 .../kafka/KafkaTestEnvironmentImpl.java         |  420 ++++
 .../src/test/resources/log4j-test.properties    |   30 +
 .../src/test/resources/logback-test.xml         |   30 +
 .../flink-connector-kafka-0.8/pom.xml           |  219 ++
 .../connectors/kafka/FlinkKafkaConsumer08.java  |  398 ++++
 .../connectors/kafka/FlinkKafkaConsumer081.java |   39 +
 .../connectors/kafka/FlinkKafkaConsumer082.java |   39 +
 .../connectors/kafka/FlinkKafkaProducer.java    |   64 +
 .../connectors/kafka/FlinkKafkaProducer08.java  |  145 ++
 .../connectors/kafka/Kafka08JsonTableSink.java  |   52 +
 .../kafka/Kafka08JsonTableSource.java           |   71 +
 .../connectors/kafka/Kafka08TableSource.java    |   75 +
 .../kafka/internals/ClosableBlockingQueue.java  |  507 +++++
 .../kafka/internals/Kafka08Fetcher.java         |  481 +++++
 .../kafka/internals/KillerWatchDog.java         |   62 +
 .../kafka/internals/PartitionInfoFetcher.java   |   66 +
 .../internals/PeriodicOffsetCommitter.java      |   85 +
 .../kafka/internals/SimpleConsumerThread.java   |  504 +++++
 .../kafka/internals/ZookeeperOffsetHandler.java |  164 ++
 .../connectors/kafka/Kafka08ITCase.java         |  248 +++
 .../kafka/Kafka08JsonTableSinkTest.java         |   48 +
 .../kafka/Kafka08JsonTableSourceTest.java       |   45 +
 .../connectors/kafka/Kafka08ProducerITCase.java |   32 +
 .../connectors/kafka/KafkaConsumer08Test.java   |  139 ++
 .../connectors/kafka/KafkaLocalSystemTime.java  |   48 +
 .../connectors/kafka/KafkaProducerTest.java     |  123 ++
 .../kafka/KafkaShortRetention08ITCase.java      |   34 +
 .../kafka/KafkaTestEnvironmentImpl.java         |  401 ++++
 .../internals/ClosableBlockingQueueTest.java    |  603 ++++++
 .../src/test/resources/log4j-test.properties    |   30 +
 .../src/test/resources/logback-test.xml         |   30 +
 .../flink-connector-kafka-0.9/pom.xml           |  212 ++
 .../connectors/kafka/FlinkKafkaConsumer09.java  |  269 +++
 .../connectors/kafka/FlinkKafkaProducer09.java  |  137 ++
 .../connectors/kafka/Kafka09JsonTableSink.java  |   50 +
 .../kafka/Kafka09JsonTableSource.java           |   71 +
 .../connectors/kafka/Kafka09TableSource.java    |   75 +
 .../connectors/kafka/internal/Handover.java     |  214 ++
 .../kafka/internal/Kafka09Fetcher.java          |  241 +++
 .../kafka/internal/KafkaConsumerCallBridge.java |   41 +
 .../kafka/internal/KafkaConsumerThread.java     |  332 +++
 .../src/main/resources/log4j.properties         |   29 +
 .../connectors/kafka/Kafka09FetcherTest.java    |  482 +++++
 .../connectors/kafka/Kafka09ITCase.java         |  129 ++
 .../kafka/Kafka09JsonTableSinkTest.java         |   48 +
 .../kafka/Kafka09JsonTableSourceTest.java       |   45 +
 .../connectors/kafka/Kafka09ProducerITCase.java |   32 +
 .../kafka/Kafka09SecuredRunITCase.java          |   62 +
 .../connectors/kafka/KafkaProducerTest.java     |  126 ++
 .../kafka/KafkaTestEnvironmentImpl.java         |  439 ++++
 .../connectors/kafka/internal/HandoverTest.java |  387 ++++
 .../src/test/resources/log4j-test.properties    |   32 +
 .../src/test/resources/logback-test.xml         |   30 +
 .../flink-connector-kafka-base/pom.xml          |  212 ++
 .../kafka/FlinkKafkaConsumerBase.java           |  552 +++++
 .../kafka/FlinkKafkaProducerBase.java           |  386 ++++
 .../connectors/kafka/KafkaJsonTableSink.java    |   47 +
 .../connectors/kafka/KafkaJsonTableSource.java  |   97 +
 .../connectors/kafka/KafkaTableSink.java        |  127 ++
 .../connectors/kafka/KafkaTableSource.java      |  155 ++
 .../kafka/internals/AbstractFetcher.java        |  552 +++++
 .../kafka/internals/ExceptionProxy.java         |  125 ++
 .../kafka/internals/KafkaTopicPartition.java    |  120 ++
 .../internals/KafkaTopicPartitionLeader.java    |   98 +
 .../internals/KafkaTopicPartitionState.java     |  118 ++
 ...picPartitionStateWithPeriodicWatermarks.java |   71 +
 ...cPartitionStateWithPunctuatedWatermarks.java |   84 +
 .../connectors/kafka/internals/TypeUtil.java    |   38 +
 .../internals/metrics/KafkaMetricWrapper.java   |   37 +
 .../kafka/partitioner/FixedPartitioner.java     |   76 +
 .../kafka/partitioner/KafkaPartitioner.java     |   41 +
 .../JSONDeserializationSchema.java              |   46 +
 .../JSONKeyValueDeserializationSchema.java      |   72 +
 .../JsonRowDeserializationSchema.java           |  135 ++
 .../JsonRowSerializationSchema.java             |   70 +
 .../KeyedDeserializationSchema.java             |   52 +
 .../KeyedDeserializationSchemaWrapper.java      |   51 +
 .../serialization/KeyedSerializationSchema.java |   55 +
 .../KeyedSerializationSchemaWrapper.java        |   48 +
 ...eInformationKeyValueSerializationSchema.java |  196 ++
 .../kafka/FlinkKafkaConsumerBaseTest.java       |  416 ++++
 .../kafka/FlinkKafkaProducerBaseTest.java       |  288 +++
 .../kafka/JSONDeserializationSchemaTest.java    |   41 +
 .../JSONKeyValueDeserializationSchemaTest.java  |   68 +
 .../kafka/JsonRowDeserializationSchemaTest.java |  124 ++
 .../kafka/JsonRowSerializationSchemaTest.java   |   98 +
 .../KafkaConsumerPartitionAssignmentTest.java   |  269 +++
 .../connectors/kafka/KafkaConsumerTestBase.java | 2006 ++++++++++++++++++
 .../connectors/kafka/KafkaProducerTestBase.java |  193 ++
 .../kafka/KafkaShortRetentionTestBase.java      |  291 +++
 .../kafka/KafkaTableSinkTestBase.java           |  106 +
 .../kafka/KafkaTableSourceTestBase.java         |   77 +
 .../connectors/kafka/KafkaTestBase.java         |  203 ++
 .../connectors/kafka/KafkaTestEnvironment.java  |  112 +
 .../connectors/kafka/TestFixedPartitioner.java  |  104 +
 .../AbstractFetcherTimestampsTest.java          |  320 +++
 .../internals/KafkaTopicPartitionTest.java      |   57 +
 .../kafka/testutils/DataGenerators.java         |  227 ++
 .../kafka/testutils/FailingIdentityMapper.java  |  115 +
 .../testutils/FakeStandardProducerConfig.java   |   34 +
 .../testutils/JobManagerCommunicationUtils.java |  120 ++
 .../testutils/PartitionValidatingMapper.java    |   53 +
 .../kafka/testutils/ThrottledMapper.java        |   44 +
 .../kafka/testutils/Tuple2Partitioner.java      |   48 +
 .../testutils/ValidatingExactlyOnceSink.java    |   82 +
 .../testutils/ZooKeeperStringSerializer.java    |   51 +
 .../src/test/resources/log4j-test.properties    |   29 +
 .../src/test/resources/logback-test.xml         |   30 +
 .../flink-connector-kinesis/pom.xml             |  164 ++
 .../kinesis/FlinkKinesisConsumer.java           |  304 +++
 .../kinesis/FlinkKinesisProducer.java           |  292 +++
 .../connectors/kinesis/KinesisPartitioner.java  |   49 +
 .../kinesis/config/AWSConfigConstants.java      |   70 +
 .../kinesis/config/ConsumerConfigConstants.java |  138 ++
 .../kinesis/config/ProducerConfigConstants.java |   33 +
 .../kinesis/examples/ConsumeFromKinesis.java    |   54 +
 .../kinesis/examples/ProduceIntoKinesis.java    |   77 +
 .../kinesis/internals/KinesisDataFetcher.java   |  679 ++++++
 .../kinesis/internals/ShardConsumer.java        |  287 +++
 .../kinesis/model/KinesisStreamShard.java       |  133 ++
 .../kinesis/model/KinesisStreamShardState.java  |   71 +
 .../kinesis/model/SentinelSequenceNumber.java   |   51 +
 .../kinesis/model/SequenceNumber.java           |  104 +
 .../kinesis/proxy/GetShardListResult.java       |   75 +
 .../connectors/kinesis/proxy/KinesisProxy.java  |  338 +++
 .../kinesis/proxy/KinesisProxyInterface.java    |   69 +
 .../KinesisDeserializationSchema.java           |   57 +
 .../KinesisDeserializationSchemaWrapper.java    |   57 +
 .../KinesisSerializationSchema.java             |   45 +
 .../connectors/kinesis/util/AWSUtil.java        |  130 ++
 .../kinesis/util/KinesisConfigUtil.java         |  218 ++
 .../src/main/resources/log4j.properties         |   27 +
 .../kinesis/FlinkKinesisConsumerTest.java       |  472 +++++
 .../internals/KinesisDataFetcherTest.java       |  510 +++++
 .../kinesis/internals/ShardConsumerTest.java    |  122 ++
 .../manualtests/ManualConsumerProducerTest.java |  121 ++
 .../manualtests/ManualExactlyOnceTest.java      |  147 ++
 ...nualExactlyOnceWithStreamReshardingTest.java |  247 +++
 .../kinesis/manualtests/ManualProducerTest.java |   91 +
 .../ExactlyOnceValidatingConsumerThread.java    |  155 ++
 .../testutils/FakeKinesisBehavioursFactory.java |  262 +++
 .../KinesisEventsGeneratorProducerThread.java   |  118 ++
 .../testutils/KinesisShardIdGenerator.java      |   25 +
 .../testutils/TestableFlinkKinesisConsumer.java |   60 +
 .../testutils/TestableKinesisDataFetcher.java   |  122 ++
 flink-connectors/flink-connector-nifi/pom.xml   |   89 +
 .../connectors/nifi/NiFiDataPacket.java         |   39 +
 .../connectors/nifi/NiFiDataPacketBuilder.java  |   34 +
 .../streaming/connectors/nifi/NiFiSink.java     |   74 +
 .../streaming/connectors/nifi/NiFiSource.java   |  155 ++
 .../connectors/nifi/StandardNiFiDataPacket.java |   46 +
 .../nifi/examples/NiFiSinkTopologyExample.java  |   55 +
 .../examples/NiFiSourceTopologyExample.java     |   58 +
 .../src/test/resources/NiFi_Flink.xml           |   16 +
 .../flink-connector-rabbitmq/pom.xml            |   60 +
 .../streaming/connectors/rabbitmq/RMQSink.java  |  142 ++
 .../connectors/rabbitmq/RMQSource.java          |  243 +++
 .../rabbitmq/common/RMQConnectionConfig.java    |  448 ++++
 .../connectors/rabbitmq/RMQSourceTest.java      |  419 ++++
 .../common/RMQConnectionConfigTest.java         |   69 +
 .../connectors/rabbitmq/common/RMQSinkTest.java |  125 ++
 flink-connectors/flink-connector-redis/pom.xml  |   79 +
 .../streaming/connectors/redis/RedisSink.java   |  188 ++
 .../common/config/FlinkJedisClusterConfig.java  |  187 ++
 .../common/config/FlinkJedisConfigBase.java     |   90 +
 .../common/config/FlinkJedisPoolConfig.java     |  224 ++
 .../common/config/FlinkJedisSentinelConfig.java |  259 +++
 .../common/container/RedisClusterContainer.java |  171 ++
 .../container/RedisCommandsContainer.java       |  115 +
 .../RedisCommandsContainerBuilder.java          |  116 +
 .../redis/common/container/RedisContainer.java  |  252 +++
 .../redis/common/mapper/RedisCommand.java       |   86 +
 .../common/mapper/RedisCommandDescription.java  |   94 +
 .../redis/common/mapper/RedisDataType.java      |   66 +
 .../redis/common/mapper/RedisMapper.java        |   66 +
 .../connectors/redis/RedisITCaseBase.java       |   45 +
 .../redis/RedisSentinelClusterTest.java         |  100 +
 .../connectors/redis/RedisSinkITCase.java       |  233 ++
 .../redis/RedisSinkPublishITCase.java           |  137 ++
 .../connectors/redis/RedisSinkTest.java         |  144 ++
 .../common/config/FlinkJedisConfigBaseTest.java |   50 +
 .../common/config/JedisClusterConfigTest.java   |   49 +
 .../common/config/JedisPoolConfigTest.java      |   29 +
 .../common/config/JedisSentinelConfigTest.java  |   49 +
 .../mapper/RedisDataTypeDescriptionTest.java    |   41 +
 .../flink-connector-twitter/pom.xml             |   96 +
 .../connectors/twitter/TwitterSource.java       |  217 ++
 .../flink-hadoop-compatibility/pom.xml          |  182 ++
 .../api/java/typeutils/WritableTypeInfo.java    |  154 ++
 .../typeutils/runtime/WritableComparator.java   |  188 ++
 .../typeutils/runtime/WritableSerializer.java   |  152 ++
 .../flink/hadoopcompatibility/HadoopInputs.java |  118 ++
 .../flink/hadoopcompatibility/HadoopUtils.java  |   52 +
 .../mapred/HadoopMapFunction.java               |  133 ++
 .../mapred/HadoopReduceCombineFunction.java     |  168 ++
 .../mapred/HadoopReduceFunction.java            |  142 ++
 .../mapred/wrapper/HadoopOutputCollector.java   |   59 +
 .../wrapper/HadoopTupleUnwrappingIterator.java  |   94 +
 .../scala/HadoopInputs.scala                    |  143 ++
 .../java/typeutils/WritableExtractionTest.java  |  206 ++
 .../java/typeutils/WritableInfoParserTest.java  |   84 +
 .../java/typeutils/WritableTypeInfoTest.java    |   72 +
 .../typeutils/runtime/StringArrayWritable.java  |   83 +
 .../runtime/WritableComparatorTest.java         |   53 +
 .../runtime/WritableComparatorUUIDTest.java     |   46 +
 .../api/java/typeutils/runtime/WritableID.java  |   78 +
 .../runtime/WritableSerializerTest.java         |   50 +
 .../runtime/WritableSerializerUUIDTest.java     |   50 +
 .../hadoopcompatibility/HadoopUtilsTest.java    |   34 +
 .../mapred/HadoopMapFunctionITCase.java         |  182 ++
 .../mapred/HadoopMapredITCase.java              |   47 +
 .../HadoopReduceCombineFunctionITCase.java      |  265 +++
 .../mapred/HadoopReduceFunctionITCase.java      |  213 ++
 .../mapred/HadoopTestData.java                  |   62 +
 .../example/HadoopMapredCompatWordCount.java    |  133 ++
 .../HadoopTupleUnwrappingIteratorTest.java      |  139 ++
 .../mapreduce/HadoopInputOutputITCase.java      |   47 +
 .../mapreduce/example/WordCount.java            |  119 ++
 .../src/test/resources/log4j-test.properties    |   27 +
 .../src/test/resources/logback-test.xml         |   29 +
 flink-connectors/flink-hbase/pom.xml            |  264 +++
 .../flink/addons/hbase/TableInputFormat.java    |  289 +++
 .../flink/addons/hbase/TableInputSplit.java     |   89 +
 .../hbase/HBaseTestingClusterAutostarter.java   |  238 +++
 .../addons/hbase/TableInputFormatITCase.java    |  120 ++
 .../hbase/example/HBaseFlinkTestConstants.java  |   28 +
 .../addons/hbase/example/HBaseReadExample.java  |   92 +
 .../addons/hbase/example/HBaseWriteExample.java |  202 ++
 .../hbase/example/HBaseWriteStreamExample.java  |  113 +
 .../src/test/resources/log4j-test.properties    |   23 +
 flink-connectors/flink-hcatalog/pom.xml         |  182 ++
 .../flink/hcatalog/HCatInputFormatBase.java     |  410 ++++
 .../flink/hcatalog/java/HCatInputFormat.java    |  160 ++
 .../flink/hcatalog/scala/HCatInputFormat.scala  |  229 ++
 flink-connectors/flink-jdbc/pom.xml             |   66 +
 .../flink/api/java/io/jdbc/JDBCInputFormat.java |  404 ++++
 .../api/java/io/jdbc/JDBCOutputFormat.java      |  315 +++
 .../split/GenericParameterValuesProvider.java   |   44 +
 .../split/NumericBetweenParametersProvider.java |   66 +
 .../io/jdbc/split/ParameterValuesProvider.java  |   35 +
 .../flink/api/java/io/jdbc/JDBCFullTest.java    |  101 +
 .../api/java/io/jdbc/JDBCInputFormatTest.java   |  247 +++
 .../api/java/io/jdbc/JDBCOutputFormatTest.java  |  169 ++
 .../flink/api/java/io/jdbc/JDBCTestBase.java    |  183 ++
 .../src/test/resources/log4j-test.properties    |   19 +
 .../src/test/resources/logback-test.xml         |   29 +
 flink-connectors/pom.xml                        |   75 +
 .../flink-connector-cassandra/pom.xml           |  179 --
 .../cassandra/CassandraInputFormat.java         |  131 --
 .../cassandra/CassandraOutputFormat.java        |  125 --
 .../cassandra/CassandraCommitter.java           |  151 --
 .../connectors/cassandra/CassandraPojoSink.java |   67 -
 .../connectors/cassandra/CassandraSink.java     |  329 ---
 .../connectors/cassandra/CassandraSinkBase.java |   98 -
 .../cassandra/CassandraTupleSink.java           |   59 -
 .../cassandra/CassandraTupleWriteAheadSink.java |  159 --
 .../connectors/cassandra/ClusterBuilder.java    |   43 -
 .../cassandra/example/BatchExample.java         |   77 -
 .../cassandra/CassandraConnectorITCase.java     |  440 ----
 .../CassandraTupleWriteAheadSinkTest.java       |  127 --
 .../streaming/connectors/cassandra/Pojo.java    |   65 -
 .../example/CassandraPojoSinkExample.java       |   62 -
 .../example/CassandraTupleSinkExample.java      |   62 -
 .../CassandraTupleWriteAheadSinkExample.java    |   96 -
 .../connectors/cassandra/example/Message.java   |   56 -
 .../src/test/resources/cassandra.yaml           |   43 -
 .../src/test/resources/log4j-test.properties    |   29 -
 .../flink-connector-elasticsearch/pom.xml       |   90 -
 .../elasticsearch/ElasticsearchSink.java        |  315 ---
 .../elasticsearch/IndexRequestBuilder.java      |   66 -
 .../elasticsearch/ElasticsearchSinkITCase.java  |  205 --
 .../examples/ElasticsearchExample.java          |   80 -
 .../src/test/resources/log4j-test.properties    |   27 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-elasticsearch2/pom.xml      |   83 -
 .../elasticsearch2/BulkProcessorIndexer.java    |   35 -
 .../elasticsearch2/ElasticsearchSink.java       |  257 ---
 .../ElasticsearchSinkFunction.java              |   60 -
 .../elasticsearch2/RequestIndexer.java          |   25 -
 .../elasticsearch2/ElasticsearchSinkITCase.java |  233 --
 .../examples/ElasticsearchExample.java          |   90 -
 .../src/test/resources/log4j-test.properties    |   27 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-filesystem/pom.xml          |  163 --
 .../connectors/fs/AvroKeyValueSinkWriter.java   |  309 ---
 .../flink/streaming/connectors/fs/Bucketer.java |   55 -
 .../flink/streaming/connectors/fs/Clock.java    |   33 -
 .../connectors/fs/DateTimeBucketer.java         |  126 --
 .../connectors/fs/NonRollingBucketer.java       |   47 -
 .../streaming/connectors/fs/RollingSink.java    |  916 --------
 .../connectors/fs/SequenceFileWriter.java       |  151 --
 .../connectors/fs/StreamWriterBase.java         |  152 --
 .../streaming/connectors/fs/StringWriter.java   |   86 -
 .../streaming/connectors/fs/SystemClock.java    |   29 -
 .../flink/streaming/connectors/fs/Writer.java   |   73 -
 .../fs/bucketing/BasePathBucketer.java          |   39 -
 .../connectors/fs/bucketing/Bucketer.java       |   47 -
 .../connectors/fs/bucketing/BucketingSink.java  | 1082 ----------
 .../fs/bucketing/DateTimeBucketer.java          |  102 -
 .../src/main/resources/log4j.properties         |   27 -
 .../fs/RollingSinkFaultToleranceITCase.java     |  300 ---
 .../connectors/fs/RollingSinkITCase.java        |  991 ---------
 .../connectors/fs/RollingSinkSecuredITCase.java |  252 ---
 .../BucketingSinkFaultToleranceITCase.java      |  297 ---
 .../fs/bucketing/BucketingSinkTest.java         |  867 --------
 .../src/test/resources/log4j-test.properties    |   29 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-flume/pom.xml               |  175 --
 .../streaming/connectors/flume/FlumeSink.java   |  141 --
 .../flink-connector-kafka-0.10/pom.xml          |  205 --
 .../connectors/kafka/FlinkKafkaConsumer010.java |  153 --
 .../connectors/kafka/FlinkKafkaProducer010.java |  398 ----
 .../kafka/Kafka010JsonTableSource.java          |   71 -
 .../connectors/kafka/Kafka010TableSource.java   |   75 -
 .../kafka/internal/Kafka010Fetcher.java         |  104 -
 .../internal/KafkaConsumerCallBridge010.java    |   40 -
 .../src/main/resources/log4j.properties         |   29 -
 .../connectors/kafka/Kafka010FetcherTest.java   |  484 -----
 .../connectors/kafka/Kafka010ITCase.java        |  313 ---
 .../kafka/Kafka010ProducerITCase.java           |   33 -
 .../kafka/KafkaTestEnvironmentImpl.java         |  420 ----
 .../src/test/resources/log4j-test.properties    |   30 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-kafka-0.8/pom.xml           |  219 --
 .../connectors/kafka/FlinkKafkaConsumer08.java  |  398 ----
 .../connectors/kafka/FlinkKafkaConsumer081.java |   39 -
 .../connectors/kafka/FlinkKafkaConsumer082.java |   39 -
 .../connectors/kafka/FlinkKafkaProducer.java    |   64 -
 .../connectors/kafka/FlinkKafkaProducer08.java  |  145 --
 .../connectors/kafka/Kafka08JsonTableSink.java  |   52 -
 .../kafka/Kafka08JsonTableSource.java           |   71 -
 .../connectors/kafka/Kafka08TableSource.java    |   75 -
 .../kafka/internals/ClosableBlockingQueue.java  |  507 -----
 .../kafka/internals/Kafka08Fetcher.java         |  481 -----
 .../kafka/internals/KillerWatchDog.java         |   62 -
 .../kafka/internals/PartitionInfoFetcher.java   |   66 -
 .../internals/PeriodicOffsetCommitter.java      |   85 -
 .../kafka/internals/SimpleConsumerThread.java   |  504 -----
 .../kafka/internals/ZookeeperOffsetHandler.java |  164 --
 .../connectors/kafka/Kafka08ITCase.java         |  248 ---
 .../kafka/Kafka08JsonTableSinkTest.java         |   48 -
 .../kafka/Kafka08JsonTableSourceTest.java       |   45 -
 .../connectors/kafka/Kafka08ProducerITCase.java |   32 -
 .../connectors/kafka/KafkaConsumer08Test.java   |  139 --
 .../connectors/kafka/KafkaLocalSystemTime.java  |   48 -
 .../connectors/kafka/KafkaProducerTest.java     |  123 --
 .../kafka/KafkaShortRetention08ITCase.java      |   34 -
 .../kafka/KafkaTestEnvironmentImpl.java         |  401 ----
 .../internals/ClosableBlockingQueueTest.java    |  603 ------
 .../src/test/resources/log4j-test.properties    |   30 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-kafka-0.9/pom.xml           |  212 --
 .../connectors/kafka/FlinkKafkaConsumer09.java  |  269 ---
 .../connectors/kafka/FlinkKafkaProducer09.java  |  137 --
 .../connectors/kafka/Kafka09JsonTableSink.java  |   50 -
 .../kafka/Kafka09JsonTableSource.java           |   71 -
 .../connectors/kafka/Kafka09TableSource.java    |   75 -
 .../connectors/kafka/internal/Handover.java     |  214 --
 .../kafka/internal/Kafka09Fetcher.java          |  241 ---
 .../kafka/internal/KafkaConsumerCallBridge.java |   41 -
 .../kafka/internal/KafkaConsumerThread.java     |  332 ---
 .../src/main/resources/log4j.properties         |   29 -
 .../connectors/kafka/Kafka09FetcherTest.java    |  482 -----
 .../connectors/kafka/Kafka09ITCase.java         |  129 --
 .../kafka/Kafka09JsonTableSinkTest.java         |   48 -
 .../kafka/Kafka09JsonTableSourceTest.java       |   45 -
 .../connectors/kafka/Kafka09ProducerITCase.java |   32 -
 .../kafka/Kafka09SecuredRunITCase.java          |   62 -
 .../connectors/kafka/KafkaProducerTest.java     |  126 --
 .../kafka/KafkaTestEnvironmentImpl.java         |  439 ----
 .../connectors/kafka/internal/HandoverTest.java |  387 ----
 .../src/test/resources/log4j-test.properties    |   32 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-kafka-base/pom.xml          |  212 --
 .../kafka/FlinkKafkaConsumerBase.java           |  552 -----
 .../kafka/FlinkKafkaProducerBase.java           |  386 ----
 .../connectors/kafka/KafkaJsonTableSink.java    |   47 -
 .../connectors/kafka/KafkaJsonTableSource.java  |   97 -
 .../connectors/kafka/KafkaTableSink.java        |  127 --
 .../connectors/kafka/KafkaTableSource.java      |  155 --
 .../kafka/internals/AbstractFetcher.java        |  552 -----
 .../kafka/internals/ExceptionProxy.java         |  125 --
 .../kafka/internals/KafkaTopicPartition.java    |  120 --
 .../internals/KafkaTopicPartitionLeader.java    |   98 -
 .../internals/KafkaTopicPartitionState.java     |  118 --
 ...picPartitionStateWithPeriodicWatermarks.java |   71 -
 ...cPartitionStateWithPunctuatedWatermarks.java |   84 -
 .../connectors/kafka/internals/TypeUtil.java    |   38 -
 .../internals/metrics/KafkaMetricWrapper.java   |   37 -
 .../kafka/partitioner/FixedPartitioner.java     |   76 -
 .../kafka/partitioner/KafkaPartitioner.java     |   41 -
 .../JSONDeserializationSchema.java              |   46 -
 .../JSONKeyValueDeserializationSchema.java      |   72 -
 .../JsonRowDeserializationSchema.java           |  135 --
 .../JsonRowSerializationSchema.java             |   70 -
 .../KeyedDeserializationSchema.java             |   52 -
 .../KeyedDeserializationSchemaWrapper.java      |   51 -
 .../serialization/KeyedSerializationSchema.java |   55 -
 .../KeyedSerializationSchemaWrapper.java        |   48 -
 ...eInformationKeyValueSerializationSchema.java |  196 --
 .../kafka/FlinkKafkaConsumerBaseTest.java       |  416 ----
 .../kafka/FlinkKafkaProducerBaseTest.java       |  288 ---
 .../kafka/JSONDeserializationSchemaTest.java    |   41 -
 .../JSONKeyValueDeserializationSchemaTest.java  |   68 -
 .../kafka/JsonRowDeserializationSchemaTest.java |  124 --
 .../kafka/JsonRowSerializationSchemaTest.java   |   98 -
 .../KafkaConsumerPartitionAssignmentTest.java   |  269 ---
 .../connectors/kafka/KafkaConsumerTestBase.java | 2006 ------------------
 .../connectors/kafka/KafkaProducerTestBase.java |  193 --
 .../kafka/KafkaShortRetentionTestBase.java      |  291 ---
 .../kafka/KafkaTableSinkTestBase.java           |  106 -
 .../kafka/KafkaTableSourceTestBase.java         |   77 -
 .../connectors/kafka/KafkaTestBase.java         |  203 --
 .../connectors/kafka/KafkaTestEnvironment.java  |  112 -
 .../connectors/kafka/TestFixedPartitioner.java  |  104 -
 .../AbstractFetcherTimestampsTest.java          |  320 ---
 .../internals/KafkaTopicPartitionTest.java      |   57 -
 .../kafka/testutils/DataGenerators.java         |  227 --
 .../kafka/testutils/FailingIdentityMapper.java  |  115 -
 .../testutils/FakeStandardProducerConfig.java   |   34 -
 .../testutils/JobManagerCommunicationUtils.java |  120 --
 .../testutils/PartitionValidatingMapper.java    |   53 -
 .../kafka/testutils/ThrottledMapper.java        |   44 -
 .../kafka/testutils/Tuple2Partitioner.java      |   48 -
 .../testutils/ValidatingExactlyOnceSink.java    |   82 -
 .../testutils/ZooKeeperStringSerializer.java    |   51 -
 .../src/test/resources/log4j-test.properties    |   29 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-kinesis/pom.xml             |  164 --
 .../kinesis/FlinkKinesisConsumer.java           |  304 ---
 .../kinesis/FlinkKinesisProducer.java           |  292 ---
 .../connectors/kinesis/KinesisPartitioner.java  |   49 -
 .../kinesis/config/AWSConfigConstants.java      |   70 -
 .../kinesis/config/ConsumerConfigConstants.java |  138 --
 .../kinesis/config/ProducerConfigConstants.java |   33 -
 .../kinesis/examples/ConsumeFromKinesis.java    |   54 -
 .../kinesis/examples/ProduceIntoKinesis.java    |   77 -
 .../kinesis/internals/KinesisDataFetcher.java   |  679 ------
 .../kinesis/internals/ShardConsumer.java        |  287 ---
 .../kinesis/model/KinesisStreamShard.java       |  133 --
 .../kinesis/model/KinesisStreamShardState.java  |   71 -
 .../kinesis/model/SentinelSequenceNumber.java   |   51 -
 .../kinesis/model/SequenceNumber.java           |  104 -
 .../kinesis/proxy/GetShardListResult.java       |   75 -
 .../connectors/kinesis/proxy/KinesisProxy.java  |  338 ---
 .../kinesis/proxy/KinesisProxyInterface.java    |   69 -
 .../KinesisDeserializationSchema.java           |   57 -
 .../KinesisDeserializationSchemaWrapper.java    |   57 -
 .../KinesisSerializationSchema.java             |   45 -
 .../connectors/kinesis/util/AWSUtil.java        |  130 --
 .../kinesis/util/KinesisConfigUtil.java         |  218 --
 .../src/main/resources/log4j.properties         |   27 -
 .../kinesis/FlinkKinesisConsumerTest.java       |  472 -----
 .../internals/KinesisDataFetcherTest.java       |  510 -----
 .../kinesis/internals/ShardConsumerTest.java    |  122 --
 .../manualtests/ManualConsumerProducerTest.java |  121 --
 .../manualtests/ManualExactlyOnceTest.java      |  147 --
 ...nualExactlyOnceWithStreamReshardingTest.java |  247 ---
 .../kinesis/manualtests/ManualProducerTest.java |   91 -
 .../ExactlyOnceValidatingConsumerThread.java    |  155 --
 .../testutils/FakeKinesisBehavioursFactory.java |  262 ---
 .../KinesisEventsGeneratorProducerThread.java   |  118 --
 .../testutils/KinesisShardIdGenerator.java      |   25 -
 .../testutils/TestableFlinkKinesisConsumer.java |   60 -
 .../testutils/TestableKinesisDataFetcher.java   |  122 --
 .../flink-connector-nifi/pom.xml                |   89 -
 .../connectors/nifi/NiFiDataPacket.java         |   39 -
 .../connectors/nifi/NiFiDataPacketBuilder.java  |   34 -
 .../streaming/connectors/nifi/NiFiSink.java     |   74 -
 .../streaming/connectors/nifi/NiFiSource.java   |  155 --
 .../connectors/nifi/StandardNiFiDataPacket.java |   46 -
 .../nifi/examples/NiFiSinkTopologyExample.java  |   55 -
 .../examples/NiFiSourceTopologyExample.java     |   58 -
 .../src/test/resources/NiFi_Flink.xml           |   16 -
 .../flink-connector-rabbitmq/pom.xml            |   60 -
 .../streaming/connectors/rabbitmq/RMQSink.java  |  142 --
 .../connectors/rabbitmq/RMQSource.java          |  243 ---
 .../rabbitmq/common/RMQConnectionConfig.java    |  448 ----
 .../connectors/rabbitmq/RMQSourceTest.java      |  419 ----
 .../common/RMQConnectionConfigTest.java         |   69 -
 .../connectors/rabbitmq/common/RMQSinkTest.java |  125 --
 .../flink-connector-redis/pom.xml               |   79 -
 .../streaming/connectors/redis/RedisSink.java   |  188 --
 .../common/config/FlinkJedisClusterConfig.java  |  187 --
 .../common/config/FlinkJedisConfigBase.java     |   90 -
 .../common/config/FlinkJedisPoolConfig.java     |  224 --
 .../common/config/FlinkJedisSentinelConfig.java |  259 ---
 .../common/container/RedisClusterContainer.java |  171 --
 .../container/RedisCommandsContainer.java       |  115 -
 .../RedisCommandsContainerBuilder.java          |  116 -
 .../redis/common/container/RedisContainer.java  |  252 ---
 .../redis/common/mapper/RedisCommand.java       |   86 -
 .../common/mapper/RedisCommandDescription.java  |   94 -
 .../redis/common/mapper/RedisDataType.java      |   66 -
 .../redis/common/mapper/RedisMapper.java        |   66 -
 .../connectors/redis/RedisITCaseBase.java       |   45 -
 .../redis/RedisSentinelClusterTest.java         |  100 -
 .../connectors/redis/RedisSinkITCase.java       |  233 --
 .../redis/RedisSinkPublishITCase.java           |  137 --
 .../connectors/redis/RedisSinkTest.java         |  144 --
 .../common/config/FlinkJedisConfigBaseTest.java |   50 -
 .../common/config/JedisClusterConfigTest.java   |   49 -
 .../common/config/JedisPoolConfigTest.java      |   29 -
 .../common/config/JedisSentinelConfigTest.java  |   49 -
 .../mapper/RedisDataTypeDescriptionTest.java    |   41 -
 .../flink-connector-twitter/pom.xml             |   96 -
 .../connectors/twitter/TwitterSource.java       |  217 --
 flink-streaming-connectors/pom.xml              |   70 -
 pom.xml                                         |    9 +-
 687 files changed, 53344 insertions(+), 53385 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index e18629d..6cbccc1 100644
--- a/.gitignore
+++ b/.gitignore
@@ -17,7 +17,7 @@ tmp
 *.log
 .DS_Store
 build-target
-flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/generated/
+flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/generated/
 flink-runtime-web/web-dashboard/assets/fonts/
 flink-runtime-web/web-dashboard/node_modules/
 flink-runtime-web/web-dashboard/bower_components/

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/pom.xml b/flink-batch-connectors/flink-avro/pom.xml
deleted file mode 100644
index 1161173..0000000
--- a/flink-batch-connectors/flink-avro/pom.xml
+++ /dev/null
@@ -1,216 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-	
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-batch-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-avro_2.10</artifactId>
-	<name>flink-avro</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-java</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.avro</groupId>
-			<artifactId>avro</artifactId>
-			<!-- version is derived from base module -->
-		</dependency>
-
-		<!-- test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils-junit</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<artifactId>maven-assembly-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>create-test-dependency</id>
-						<phase>process-test-classes</phase>
-						<goals>
-							<goal>single</goal>
-						</goals>
-						<configuration>
-							<archive>
-								<manifest>
-									<mainClass>org.apache.flink.api.avro.testjar.AvroExternalJarProgram</mainClass>
-								</manifest>
-							</archive>
-							<finalName>maven</finalName>
-							<attach>false</attach>
-							<descriptors>
-								<descriptor>src/test/assembly/test-assembly.xml</descriptor>
-							</descriptors>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-			<!--Remove the AvroExternalJarProgram code from the test-classes directory since it musn't be in the
-			classpath when running the tests to actually test whether the user code class loader
-			is properly used.-->
-			<plugin>
-				<artifactId>maven-clean-plugin</artifactId>
-				<version>2.5</version><!--$NO-MVN-MAN-VER$-->
-				<executions>
-					<execution>
-						<id>remove-avroexternalprogram</id>
-						<phase>process-test-classes</phase>
-						<goals>
-							<goal>clean</goal>
-						</goals>
-						<configuration>
-							<excludeDefaultDirectories>true</excludeDefaultDirectories>
-							<filesets>
-								<fileset>
-									<directory>${project.build.testOutputDirectory}</directory>
-									<includes>
-										<include>**/testjar/*.class</include>
-									</includes>
-								</fileset>
-							</filesets>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-			<!-- Generate Test class from avro schema -->
-			<plugin>
-				<groupId>org.apache.avro</groupId>
-				<artifactId>avro-maven-plugin</artifactId>
-				<version>1.7.7</version>
-				<executions>
-					<execution>
-						<phase>generate-sources</phase>
-						<goals>
-							<goal>schema</goal>
-						</goals>
-						<configuration>
-							<testSourceDirectory>${project.basedir}/src/test/resources/avro</testSourceDirectory>
-							<testOutputDirectory>${project.basedir}/src/test/java/</testOutputDirectory>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-		
-		<pluginManagement>
-			<plugins>
-				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
-				<plugin>
-					<groupId>org.eclipse.m2e</groupId>
-					<artifactId>lifecycle-mapping</artifactId>
-					<version>1.0.0</version>
-					<configuration>
-						<lifecycleMappingMetadata>
-							<pluginExecutions>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>org.apache.maven.plugins</groupId>
-										<artifactId>maven-assembly-plugin</artifactId>
-										<versionRange>[2.4,)</versionRange>
-										<goals>
-											<goal>single</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>org.apache.maven.plugins</groupId>
-										<artifactId>maven-clean-plugin</artifactId>
-										<versionRange>[1,)</versionRange>
-										<goals>
-											<goal>clean</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>org.apache.avro</groupId>
-										<artifactId>avro-maven-plugin</artifactId>
-										<versionRange>[1.7.7,)</versionRange>
-										<goals>
-											<goal>schema</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-							</pluginExecutions>
-						</lifecycleMappingMetadata>
-					</configuration>
-				</plugin>
-			</plugins>
-		</pluginManagement>
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataInputDecoder.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataInputDecoder.java b/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataInputDecoder.java
deleted file mode 100644
index 59da4cb..0000000
--- a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataInputDecoder.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.avro;
-
-import java.io.DataInput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.avro.io.Decoder;
-import org.apache.avro.util.Utf8;
-
-
-public class DataInputDecoder extends Decoder {
-	
-	private final Utf8 stringDecoder = new Utf8();
-	
-	private DataInput in;
-	
-	public void setIn(DataInput in) {
-		this.in = in;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	// primitives
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public void readNull() {}
-	
-
-	@Override
-	public boolean readBoolean() throws IOException {
-		return in.readBoolean();
-	}
-
-	@Override
-	public int readInt() throws IOException {
-		return in.readInt();
-	}
-
-	@Override
-	public long readLong() throws IOException {
-		return in.readLong();
-	}
-
-	@Override
-	public float readFloat() throws IOException {
-		return in.readFloat();
-	}
-
-	@Override
-	public double readDouble() throws IOException {
-		return in.readDouble();
-	}
-	
-	@Override
-	public int readEnum() throws IOException {
-		return readInt();
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// bytes
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public void readFixed(byte[] bytes, int start, int length) throws IOException {
-		in.readFully(bytes, start, length);
-	}
-	
-	@Override
-	public ByteBuffer readBytes(ByteBuffer old) throws IOException {
-		int length = readInt();
-		ByteBuffer result;
-		if (old != null && length <= old.capacity() && old.hasArray()) {
-			result = old;
-			result.clear();
-		} else {
-			result = ByteBuffer.allocate(length);
-		}
-		in.readFully(result.array(), result.arrayOffset() + result.position(), length);
-		result.limit(length);
-		return result;
-	}
-	
-	
-	@Override
-	public void skipFixed(int length) throws IOException {
-		skipBytes(length);
-	}
-	
-	@Override
-	public void skipBytes() throws IOException {
-		int num = readInt();
-		skipBytes(num);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// strings
-	// --------------------------------------------------------------------------------------------
-	
-	
-	@Override
-	public Utf8 readString(Utf8 old) throws IOException {
-		int length = readInt();
-		Utf8 result = (old != null ? old : new Utf8());
-		result.setByteLength(length);
-		
-		if (length > 0) {
-			in.readFully(result.getBytes(), 0, length);
-		}
-		
-		return result;
-	}
-
-	@Override
-	public String readString() throws IOException {
-		return readString(stringDecoder).toString();
-	}
-
-	@Override
-	public void skipString() throws IOException {
-		int len = readInt();
-		skipBytes(len);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	// collection types
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public long readArrayStart() throws IOException {
-		return readVarLongCount(in);
-	}
-
-	@Override
-	public long arrayNext() throws IOException {
-		return readVarLongCount(in);
-	}
-
-	@Override
-	public long skipArray() throws IOException {
-		return readVarLongCount(in);
-	}
-
-	@Override
-	public long readMapStart() throws IOException {
-		return readVarLongCount(in);
-	}
-
-	@Override
-	public long mapNext() throws IOException {
-		return readVarLongCount(in);
-	}
-
-	@Override
-	public long skipMap() throws IOException {
-		return readVarLongCount(in);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// union
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public int readIndex() throws IOException {
-		return readInt();
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// utils
-	// --------------------------------------------------------------------------------------------
-	
-	private void skipBytes(int num) throws IOException {
-		while (num > 0) {
-			num -= in.skipBytes(num);
-		}
-	}
-	
-	public static long readVarLongCount(DataInput in) throws IOException {
-		long value = in.readUnsignedByte();
-
-		if ((value & 0x80) == 0) {
-			return value;
-		}
-		else {
-			long curr;
-			int shift = 7;
-			value = value & 0x7f;
-			while (((curr = in.readUnsignedByte()) & 0x80) != 0){
-				value |= (curr & 0x7f) << shift;
-				shift += 7;
-			}
-			value |= curr << shift;
-			return value;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataOutputEncoder.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataOutputEncoder.java b/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataOutputEncoder.java
deleted file mode 100644
index 0102cc1..0000000
--- a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataOutputEncoder.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.avro;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.avro.io.Encoder;
-import org.apache.avro.util.Utf8;
-
-
-public final class DataOutputEncoder extends Encoder implements java.io.Serializable {
-	
-	private static final long serialVersionUID = 1L;
-	
-	private DataOutput out;
-	
-	
-	public void setOut(DataOutput out) {
-		this.out = out;
-	}
-
-
-	@Override
-	public void flush() throws IOException {}
-
-	// --------------------------------------------------------------------------------------------
-	// primitives
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public void writeNull() {}
-	
-
-	@Override
-	public void writeBoolean(boolean b) throws IOException {
-		out.writeBoolean(b);
-	}
-
-	@Override
-	public void writeInt(int n) throws IOException {
-		out.writeInt(n);
-	}
-
-	@Override
-	public void writeLong(long n) throws IOException {
-		out.writeLong(n);
-	}
-
-	@Override
-	public void writeFloat(float f) throws IOException {
-		out.writeFloat(f);
-	}
-
-	@Override
-	public void writeDouble(double d) throws IOException {
-		out.writeDouble(d);
-	}
-	
-	@Override
-	public void writeEnum(int e) throws IOException {
-		out.writeInt(e);
-	}
-	
-	
-	// --------------------------------------------------------------------------------------------
-	// bytes
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public void writeFixed(byte[] bytes, int start, int len) throws IOException {
-		out.write(bytes, start, len);
-	}
-	
-	@Override
-	public void writeBytes(byte[] bytes, int start, int len) throws IOException {
-		out.writeInt(len);
-		if (len > 0) {
-			out.write(bytes, start, len);
-		}
-	}
-	
-	@Override
-	public void writeBytes(ByteBuffer bytes) throws IOException {
-		int num = bytes.remaining();
-		out.writeInt(num);
-		
-		if (num > 0) {
-			writeFixed(bytes);
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// strings
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public void writeString(String str) throws IOException {
-		byte[] bytes = Utf8.getBytesFor(str);
-		writeBytes(bytes, 0, bytes.length);
-	}
-	
-	@Override
-	public void writeString(Utf8 utf8) throws IOException {
-		writeBytes(utf8.getBytes(), 0, utf8.getByteLength());
-		
-	}
-
-	// --------------------------------------------------------------------------------------------
-	// collection types
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public void writeArrayStart() {}
-
-	@Override
-	public void setItemCount(long itemCount) throws IOException {
-		if (itemCount > 0) {
-			writeVarLongCount(out, itemCount);
-		}
-	}
-
-	@Override
-	public void startItem() {}
-
-	@Override
-	public void writeArrayEnd() throws IOException {
-		// write a single byte 0, shortcut for a var-length long of 0
-		out.write(0);
-	}
-
-	@Override
-	public void writeMapStart() {}
-
-	@Override
-	public void writeMapEnd() throws IOException {
-		// write a single byte 0, shortcut for a var-length long of 0
-		out.write(0);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	// union
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public void writeIndex(int unionIndex) throws IOException {
-		out.writeInt(unionIndex);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// utils
-	// --------------------------------------------------------------------------------------------
-		
-	
-	public static void writeVarLongCount(DataOutput out, long val) throws IOException {
-		if (val < 0) {
-			throw new IOException("Illegal count (must be non-negative): " + val);
-		}
-		
-		while ((val & ~0x7FL) != 0) {
-			out.write(((int) val) | 0x80);
-			val >>>= 7;
-		}
-		out.write((int) val);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java b/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java
deleted file mode 100644
index 709c4f1..0000000
--- a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.avro;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.avro.file.SeekableInput;
-import org.apache.flink.core.fs.FSDataInputStream;
-
-
-/**
- * Code copy pasted from org.apache.avro.mapred.FSInput (which is Apache licensed as well)
- * 
- * The wrapper keeps track of the position in the data stream.
- */
-public class FSDataInputStreamWrapper implements Closeable, SeekableInput {
-	private final FSDataInputStream stream;
-	private long pos;
-	private long len;
-
-	public FSDataInputStreamWrapper(FSDataInputStream stream, long len) {
-		this.stream = stream;
-		this.pos = 0;
-		this.len = len;
-	}
-
-	public long length() throws IOException {
-		return this.len;
-	}
-
-	public int read(byte[] b, int off, int len) throws IOException {
-		int read;
-		read = stream.read(b, off, len);
-		pos += read;
-		return read;
-	}
-
-	public void seek(long p) throws IOException {
-		stream.seek(p);
-		pos = p;
-	}
-
-	public long tell() throws IOException {
-		return pos;
-	}
-
-	public void close() throws IOException {
-		stream.close();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java b/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
deleted file mode 100644
index 73067c1..0000000
--- a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.io.IOException;
-
-import org.apache.avro.file.DataFileReader;
-import org.apache.avro.file.SeekableInput;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.reflect.ReflectDatumReader;
-import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.flink.api.common.io.CheckpointableInputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.api.avro.FSDataInputStreamWrapper;
-import org.apache.flink.api.common.io.FileInputFormat;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.util.InstantiationUtil;
-
-/**
- * Provides a {@link FileInputFormat} for Avro records.
- *
- * @param <E>
- *            the type of the result Avro record. If you specify
- *            {@link GenericRecord} then the result will be returned as a
- *            {@link GenericRecord}, so you do not have to know the schema ahead
- *            of time.
- */
-public class AvroInputFormat<E> extends FileInputFormat<E> implements ResultTypeQueryable<E>,
-	CheckpointableInputFormat<FileInputSplit, Tuple2<Long, Long>> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(AvroInputFormat.class);
-
-	private final Class<E> avroValueType;
-	
-	private boolean reuseAvroValue = true;
-
-	private transient DataFileReader<E> dataFileReader;
-
-	private transient long end;
-
-	private transient long recordsReadSinceLastSync;
-
-	private long lastSync = -1l;
-
-	public AvroInputFormat(Path filePath, Class<E> type) {
-		super(filePath);
-		this.avroValueType = type;
-	}
-
-	/**
-	 * Sets the flag whether to reuse the Avro value instance for all records.
-	 * By default, the input format reuses the Avro value.
-	 *
-	 * @param reuseAvroValue True, if the input format should reuse the Avro value instance, false otherwise.
-	 */
-	public void setReuseAvroValue(boolean reuseAvroValue) {
-		this.reuseAvroValue = reuseAvroValue;
-	}
-
-	/**
-	 * If set, the InputFormat will only read entire files.
-	 */
-	public void setUnsplittable(boolean unsplittable) {
-		this.unsplittable = unsplittable;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// Typing
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public TypeInformation<E> getProducedType() {
-		return TypeExtractor.getForClass(this.avroValueType);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// Input Format Methods
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public void open(FileInputSplit split) throws IOException {
-		super.open(split);
-		dataFileReader = initReader(split);
-		dataFileReader.sync(split.getStart());
-		lastSync = dataFileReader.previousSync();
-	}
-
-	private DataFileReader<E> initReader(FileInputSplit split) throws IOException {
-		DatumReader<E> datumReader;
-
-		if (org.apache.avro.generic.GenericRecord.class == avroValueType) {
-			datumReader = new GenericDatumReader<E>();
-		} else {
-			datumReader = org.apache.avro.specific.SpecificRecordBase.class.isAssignableFrom(avroValueType)
-				? new SpecificDatumReader<E>(avroValueType) : new ReflectDatumReader<E>(avroValueType);
-		}
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Opening split {}", split);
-		}
-
-		SeekableInput in = new FSDataInputStreamWrapper(stream, split.getPath().getFileSystem().getFileStatus(split.getPath()).getLen());
-		DataFileReader<E> dataFileReader = (DataFileReader) DataFileReader.openReader(in, datumReader);
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Loaded SCHEMA: {}", dataFileReader.getSchema());
-		}
-
-		end = split.getStart() + split.getLength();
-		recordsReadSinceLastSync = 0;
-		return dataFileReader;
-	}
-
-	@Override
-	public boolean reachedEnd() throws IOException {
-		return !dataFileReader.hasNext() || dataFileReader.pastSync(end);
-	}
-
-	public long getRecordsReadFromBlock() {
-		return this.recordsReadSinceLastSync;
-	}
-
-	@Override
-	public E nextRecord(E reuseValue) throws IOException {
-		if (reachedEnd()) {
-			return null;
-		}
-
-		// if we start a new block, then register the event, and
-		// restart the counter.
-		if(dataFileReader.previousSync() != lastSync) {
-			lastSync = dataFileReader.previousSync();
-			recordsReadSinceLastSync = 0;
-		}
-		recordsReadSinceLastSync++;
-
-		if (reuseAvroValue) {
-			return dataFileReader.next(reuseValue);
-		} else {
-			if (GenericRecord.class == avroValueType) {
-				return dataFileReader.next();
-			} else {
-				return dataFileReader.next(InstantiationUtil.instantiate(avroValueType, Object.class));
-			}
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Checkpointing
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public Tuple2<Long, Long> getCurrentState() throws IOException {
-		return new Tuple2<>(this.lastSync, this.recordsReadSinceLastSync);
-	}
-
-	@Override
-	public void reopen(FileInputSplit split, Tuple2<Long, Long> state) throws IOException {
-		Preconditions.checkNotNull(split, "reopen() cannot be called on a null split.");
-		Preconditions.checkNotNull(state, "reopen() cannot be called with a null initial state.");
-
-		try {
-			this.open(split);
-		} finally {
-			if (state.f0 != -1) {
-				lastSync = state.f0;
-				recordsReadSinceLastSync = state.f1;
-			}
-		}
-
-		if (lastSync != -1) {
-			// open and read until the record we were before
-			// the checkpoint and discard the values
-			dataFileReader.seek(lastSync);
-			for(int i = 0; i < recordsReadSinceLastSync; i++) {
-				dataFileReader.next(null);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java b/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
deleted file mode 100644
index 600d1e5..0000000
--- a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.api.java.io;
-
-import org.apache.avro.Schema;
-import org.apache.avro.file.CodecFactory;
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.reflect.ReflectData;
-import org.apache.avro.reflect.ReflectDatumWriter;
-import org.apache.avro.specific.SpecificDatumWriter;
-import org.apache.flink.api.common.io.FileOutputFormat;
-import org.apache.flink.core.fs.Path;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-public class AvroOutputFormat<E> extends FileOutputFormat<E> implements Serializable {
-
-	/**
-	 * Wrapper which encapsulates the supported codec and a related serialization byte.
-	 */
-	public enum Codec {
-
-		NULL((byte)0, CodecFactory.nullCodec()),
-		SNAPPY((byte)1, CodecFactory.snappyCodec()),
-		BZIP2((byte)2, CodecFactory.bzip2Codec()),
-		DEFLATE((byte)3, CodecFactory.deflateCodec(CodecFactory.DEFAULT_DEFLATE_LEVEL)),
-		XZ((byte)4, CodecFactory.xzCodec(CodecFactory.DEFAULT_XZ_LEVEL));
-
-		private byte codecByte;
-
-		private CodecFactory codecFactory;
-
-		Codec(final byte codecByte, final CodecFactory codecFactory) {
-			this.codecByte = codecByte;
-			this.codecFactory = codecFactory;
-		}
-
-		private byte getCodecByte() {
-			return codecByte;
-		}
-
-		private CodecFactory getCodecFactory() {
-			return codecFactory;
-		}
-
-		private static Codec forCodecByte(byte codecByte) {
-			for (final Codec codec : Codec.values()) {
-				if (codec.getCodecByte() == codecByte) {
-					return codec;
-				}
-			}
-			throw new IllegalArgumentException("no codec for codecByte: " + codecByte);
-		}
-	}
-
-	private static final long serialVersionUID = 1L;
-
-	private final Class<E> avroValueType;
-
-	private transient Schema userDefinedSchema = null;
-
-	private transient Codec codec = null;
-	
-	private transient DataFileWriter<E> dataFileWriter;
-
-	public AvroOutputFormat(Path filePath, Class<E> type) {
-		super(filePath);
-		this.avroValueType = type;
-	}
-
-	public AvroOutputFormat(Class<E> type) {
-		this.avroValueType = type;
-	}
-
-	@Override
-	protected String getDirectoryFileName(int taskNumber) {
-		return super.getDirectoryFileName(taskNumber) + ".avro";
-	}
-
-	public void setSchema(Schema schema) {
-		this.userDefinedSchema = schema;
-	}
-
-	/**
-	 * Set avro codec for compression.
-	 *
-	 * @param codec avro codec.
-	 */
-	public void setCodec(final Codec codec) {
-		this.codec = checkNotNull(codec, "codec can not be null");
-	}
-
-	@Override
-	public void writeRecord(E record) throws IOException {
-		dataFileWriter.append(record);
-	}
-
-	@Override
-	public void open(int taskNumber, int numTasks) throws IOException {
-		super.open(taskNumber, numTasks);
-
-		DatumWriter<E> datumWriter;
-		Schema schema;
-		if (org.apache.avro.specific.SpecificRecordBase.class.isAssignableFrom(avroValueType)) {
-			datumWriter = new SpecificDatumWriter<E>(avroValueType);
-			try {
-				schema = ((org.apache.avro.specific.SpecificRecordBase)avroValueType.newInstance()).getSchema();
-			} catch (InstantiationException e) {
-				throw new RuntimeException(e.getMessage());
-			} catch (IllegalAccessException e) {
-				throw new RuntimeException(e.getMessage());
-			}
-		} else {
-			datumWriter = new ReflectDatumWriter<E>(avroValueType);
-			schema = ReflectData.get().getSchema(avroValueType);
-		}
-		dataFileWriter = new DataFileWriter<E>(datumWriter);
-		if (codec != null) {
-			dataFileWriter.setCodec(codec.getCodecFactory());
-		}
-		if (userDefinedSchema == null) {
-			dataFileWriter.create(schema, stream);
-		} else {
-			dataFileWriter.create(userDefinedSchema, stream);
-		}
-	}
-
-	private void writeObject(java.io.ObjectOutputStream out) throws IOException {
-		out.defaultWriteObject();
-
-		if (codec != null) {
-			out.writeByte(codec.getCodecByte());
-		} else {
-			out.writeByte(-1);
-		}
-
-		if(userDefinedSchema != null) {
-			byte[] json = userDefinedSchema.toString().getBytes();
-			out.writeInt(json.length);
-			out.write(json);
-		} else {
-			out.writeInt(0);
-		}
-	}
-
-	private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
-		in.defaultReadObject();
-
-		byte codecByte = in.readByte();
-		if (codecByte >= 0) {
-			setCodec(Codec.forCodecByte(codecByte));
-		}
-
-		int length = in.readInt();
-		if(length != 0) {
-			byte[] json = new byte[length];
-			in.readFully(json);
-
-			Schema schema = new Schema.Parser().parse(new String(json));
-			setSchema(schema);
-		}
-	}
-
-	@Override
-	public void close() throws IOException {
-		dataFileWriter.flush();
-		dataFileWriter.close();
-		super.close();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/assembly/test-assembly.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/assembly/test-assembly.xml b/flink-batch-connectors/flink-avro/src/test/assembly/test-assembly.xml
deleted file mode 100644
index 0f4561a..0000000
--- a/flink-batch-connectors/flink-avro/src/test/assembly/test-assembly.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-
-<assembly>
-	<id>test-jar</id>
-	<formats>
-		<format>jar</format>
-	</formats>
-	<includeBaseDirectory>false</includeBaseDirectory>
-	<fileSets>
-		<fileSet>
-			<directory>${project.build.testOutputDirectory}</directory>
-			<outputDirectory>/</outputDirectory>
-			<!--modify/add include to match your package(s) -->
-			<includes>
-				<include>org/apache/flink/api/avro/testjar/**</include>
-			</includes>
-		</fileSet>
-	</fileSets>
-</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
deleted file mode 100644
index 1030ff8..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.avro;
-
-import java.io.File;
-
-import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.client.program.PackagedProgram;
-import org.apache.flink.client.program.StandaloneClusterClient;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class AvroExternalJarProgramITCase {
-
-	private static final String JAR_FILE = "maven-test-jar.jar";
-
-	private static final String TEST_DATA_FILE = "/testdata.avro";
-
-	@Test
-	public void testExternalProgram() {
-
-		LocalFlinkMiniCluster testMiniCluster = null;
-
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
-			testMiniCluster = new LocalFlinkMiniCluster(config, false);
-			testMiniCluster.start();
-
-			String jarFile = JAR_FILE;
-			String testData = getClass().getResource(TEST_DATA_FILE).toString();
-
-			PackagedProgram program = new PackagedProgram(new File(jarFile), new String[] { testData });
-
-
-			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
-			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, testMiniCluster.getLeaderRPCPort());
-
-			ClusterClient client = new StandaloneClusterClient(config);
-
-			client.setPrintStatusDuringExecution(false);
-			client.run(program, 4);
-
-		}
-		catch (Throwable t) {
-			System.err.println(t.getMessage());
-			t.printStackTrace();
-			Assert.fail("Error during the packaged program execution: " + t.getMessage());
-		}
-		finally {
-			if (testMiniCluster != null) {
-				try {
-					testMiniCluster.stop();
-				} catch (Throwable t) {
-					// ignore
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java
deleted file mode 100644
index 3b01ccb..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.avro;
-
-import org.junit.Assert;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.avro.file.DataFileReader;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.reflect.ReflectDatumReader;
-import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.flink.api.io.avro.example.User;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.io.AvroOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-@SuppressWarnings("serial")
-public class AvroOutputFormatITCase extends JavaProgramTestBase {
-
-	public static String outputPath1;
-
-	public static String outputPath2;
-
-	public static String inputPath;
-
-	public static String userData = "alice|1|blue\n" +
-		"bob|2|red\n" +
-		"john|3|yellow\n" +
-		"walt|4|black\n";
-
-	@Override
-	protected void preSubmit() throws Exception {
-		inputPath = createTempFile("user", userData);
-		outputPath1 = getTempDirPath("avro_output1");
-		outputPath2 = getTempDirPath("avro_output2");
-	}
-
-
-	@Override
-	protected void testProgram() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<String, Integer, String>> input = env.readCsvFile(inputPath)
-			.fieldDelimiter("|")
-			.types(String.class, Integer.class, String.class);
-
-		//output the data with AvroOutputFormat for specific user type
-		DataSet<User> specificUser = input.map(new ConvertToUser());
-		AvroOutputFormat<User> avroOutputFormat = new AvroOutputFormat<User>(User.class);
-		avroOutputFormat.setCodec(AvroOutputFormat.Codec.SNAPPY); // FLINK-4771: use a codec
-		avroOutputFormat.setSchema(User.SCHEMA$); //FLINK-3304: Ensure the OF is properly serializing the schema
-		specificUser.write(avroOutputFormat, outputPath1);
-
-		//output the data with AvroOutputFormat for reflect user type
-		DataSet<ReflectiveUser> reflectiveUser = specificUser.map(new ConvertToReflective());
-		reflectiveUser.write(new AvroOutputFormat<ReflectiveUser>(ReflectiveUser.class), outputPath2);
-
-		env.execute();
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		//compare result for specific user type
-		File [] output1;
-		File file1 = asFile(outputPath1);
-		if (file1.isDirectory()) {
-			output1 = file1.listFiles();
-			// check for avro ext in dir.
-			for (File avroOutput : output1) {
-				Assert.assertTrue("Expect extension '.avro'", avroOutput.toString().endsWith(".avro"));
-			}
-		} else {
-			output1 = new File[] {file1};
-		}
-		List<String> result1 = new ArrayList<String>();
-		DatumReader<User> userDatumReader1 = new SpecificDatumReader<User>(User.class);
-		for (File avroOutput : output1) {
-
-			DataFileReader<User> dataFileReader1 = new DataFileReader<User>(avroOutput, userDatumReader1);
-			while (dataFileReader1.hasNext()) {
-				User user = dataFileReader1.next();
-				result1.add(user.getName() + "|" + user.getFavoriteNumber() + "|" + user.getFavoriteColor());
-			}
-		}
-		for (String expectedResult : userData.split("\n")) {
-			Assert.assertTrue("expected user " + expectedResult + " not found.", result1.contains(expectedResult));
-		}
-
-		//compare result for reflect user type
-		File [] output2;
-		File file2 = asFile(outputPath2);
-		if (file2.isDirectory()) {
-			output2 = file2.listFiles();
-		} else {
-			output2 = new File[] {file2};
-		}
-		List<String> result2 = new ArrayList<String>();
-		DatumReader<ReflectiveUser> userDatumReader2 = new ReflectDatumReader<ReflectiveUser>(ReflectiveUser.class);
-		for (File avroOutput : output2) {
-			DataFileReader<ReflectiveUser> dataFileReader2 = new DataFileReader<ReflectiveUser>(avroOutput, userDatumReader2);
-			while (dataFileReader2.hasNext()) {
-				ReflectiveUser user = dataFileReader2.next();
-				result2.add(user.getName() + "|" + user.getFavoriteNumber() + "|" + user.getFavoriteColor());
-			}
-		}
-		for (String expectedResult : userData.split("\n")) {
-			Assert.assertTrue("expected user " + expectedResult + " not found.", result2.contains(expectedResult));
-		}
-
-
-	}
-
-
-	public final static class ConvertToUser extends RichMapFunction<Tuple3<String, Integer, String>, User> {
-
-		@Override
-		public User map(Tuple3<String, Integer, String> value) throws Exception {
-			return new User(value.f0, value.f1, value.f2);
-		}
-	}
-
-	public final static class ConvertToReflective extends RichMapFunction<User, ReflectiveUser> {
-
-		@Override
-		public ReflectiveUser map(User value) throws Exception {
-			return new ReflectiveUser(value.getName().toString(), value.getFavoriteNumber(), value.getFavoriteColor().toString());
-		}
-	}
-
-	
-	public static class ReflectiveUser {
-		private String name;
-		private int favoriteNumber;
-		private String favoriteColor;
-
-		public ReflectiveUser() {}
-
-		public ReflectiveUser(String name, int favoriteNumber, String favoriteColor) {
-			this.name = name;
-			this.favoriteNumber = favoriteNumber;
-			this.favoriteColor = favoriteColor;
-		}
-		
-		public String getName() {
-			return this.name;
-		}
-		public String getFavoriteColor() {
-			return this.favoriteColor;
-		}
-		public int getFavoriteNumber() {
-			return this.favoriteNumber;
-		}
-	}
-}


[14/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
deleted file mode 100644
index 80ae294..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
+++ /dev/null
@@ -1,991 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.fs;
-
-import org.apache.avro.Schema;
-import org.apache.avro.Schema.Type;
-import org.apache.avro.file.DataFileConstants;
-import org.apache.avro.file.DataFileStream;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.GenericData.StringType;
-import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.commons.io.FileUtils;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichFilterFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.testutils.MultiShotLatch;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.fs.AvroKeyValueSinkWriter.AvroKeyValue;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.NetUtils;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Tests for {@link RollingSink}. These
- * tests test the different output methods as well as the rolling feature using a manual clock
- * that increases time in lockstep with element computation using latches.
- *
- * <p>
- * This only tests the rolling behaviour of the sink. There is a separate ITCase that verifies
- * exactly once behaviour.
- *
- * @deprecated should be removed with the {@link RollingSink}.
- */
-@Deprecated
-public class RollingSinkITCase extends StreamingMultipleProgramsTestBase {
-
-	protected static final Logger LOG = LoggerFactory.getLogger(RollingSinkITCase.class);
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	protected static MiniDFSCluster hdfsCluster;
-	protected static org.apache.hadoop.fs.FileSystem dfs;
-	protected static String hdfsURI;
-	protected static Configuration conf = new Configuration();
-
-	protected static File dataDir;
-
-	@BeforeClass
-	public static void createHDFS() throws IOException {
-
-		LOG.info("In RollingSinkITCase: Starting MiniDFSCluster ");
-
-		dataDir = tempFolder.newFolder();
-
-		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
-		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-		hdfsCluster = builder.build();
-
-		dfs = hdfsCluster.getFileSystem();
-
-		hdfsURI = "hdfs://"
-				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
-				+ "/";
-	}
-
-	@AfterClass
-	public static void destroyHDFS() {
-		LOG.info("In RollingSinkITCase: tearing down MiniDFSCluster ");
-		hdfsCluster.shutdown();
-	}
-
-	/**
-	 * This tests {@link StringWriter} with
-	 * non-rolling output.
-	 */
-	@Test
-	public void testNonRollingStringWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/string-non-rolling-out";
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
-				.broadcast()
-				.filter(new OddEvenFilter());
-
-		RollingSink<String> sink = new RollingSink<String>(outPath)
-				.setBucketer(new NonRollingBucketer())
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		source
-				.map(new MapFunction<Tuple2<Integer,String>, String>() {
-					private static final long serialVersionUID = 1L;
-					@Override
-					public String map(Tuple2<Integer, String> value) throws Exception {
-						return value.f1;
-					}
-				})
-				.addSink(sink);
-
-		env.execute("RollingSink String Write Test");
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
-
-		BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
-			String line = br.readLine();
-			Assert.assertEquals("message #" + i, line);
-		}
-
-		inStream.close();
-
-		inStream = dfs.open(new Path(outPath + "/part-1-0"));
-
-		br = new BufferedReader(new InputStreamReader(inStream));
-
-		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
-			String line = br.readLine();
-			Assert.assertEquals("message #" + i, line);
-		}
-
-		inStream.close();
-	}
-
-	/**
-	 * This tests {@link SequenceFileWriter}
-	 * with non-rolling output and without compression.
-	 */
-	@Test
-	public void testNonRollingSequenceFileWithoutCompressionWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/seq-no-comp-non-rolling-out";
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
-				.broadcast()
-				.filter(new OddEvenFilter());
-
-		DataStream<Tuple2<IntWritable, Text>> mapped =  source.map(new MapFunction<Tuple2<Integer,String>, Tuple2<IntWritable, Text>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Tuple2<IntWritable, Text> map(Tuple2<Integer, String> value) throws Exception {
-				return Tuple2.of(new IntWritable(value.f0), new Text(value.f1));
-			}
-		});
-
-
-		RollingSink<Tuple2<IntWritable, Text>> sink = new RollingSink<Tuple2<IntWritable, Text>>(outPath)
-				.setWriter(new SequenceFileWriter<IntWritable, Text>())
-				.setBucketer(new NonRollingBucketer())
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		mapped.addSink(sink);
-
-		env.execute("RollingSink String Write Test");
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
-
-		SequenceFile.Reader reader = new SequenceFile.Reader(inStream,
-				1000,
-				0,
-				100000,
-				new Configuration());
-
-		IntWritable intWritable = new IntWritable();
-		Text txt = new Text();
-
-		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
-			reader.next(intWritable, txt);
-			Assert.assertEquals(i, intWritable.get());
-			Assert.assertEquals("message #" + i, txt.toString());
-		}
-
-		reader.close();
-		inStream.close();
-
-		inStream = dfs.open(new Path(outPath + "/part-1-0"));
-
-		reader = new SequenceFile.Reader(inStream,
-				1000,
-				0,
-				100000,
-				new Configuration());
-
-		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
-			reader.next(intWritable, txt);
-			Assert.assertEquals(i, intWritable.get());
-			Assert.assertEquals("message #" + i, txt.toString());
-		}
-
-		reader.close();
-		inStream.close();
-	}
-
-	/**
-	 * This tests {@link SequenceFileWriter}
-	 * with non-rolling output but with compression.
-	 */
-	@Test
-	public void testNonRollingSequenceFileWithCompressionWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/seq-non-rolling-out";
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
-				.broadcast()
-				.filter(new OddEvenFilter());
-
-		DataStream<Tuple2<IntWritable, Text>> mapped =  source.map(new MapFunction<Tuple2<Integer,String>, Tuple2<IntWritable, Text>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Tuple2<IntWritable, Text> map(Tuple2<Integer, String> value) throws Exception {
-				return Tuple2.of(new IntWritable(value.f0), new Text(value.f1));
-			}
-		});
-
-
-		RollingSink<Tuple2<IntWritable, Text>> sink = new RollingSink<Tuple2<IntWritable, Text>>(outPath)
-				.setWriter(new SequenceFileWriter<IntWritable, Text>("Default", SequenceFile.CompressionType.BLOCK))
-				.setBucketer(new NonRollingBucketer())
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		mapped.addSink(sink);
-
-		env.execute("RollingSink String Write Test");
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
-
-		SequenceFile.Reader reader = new SequenceFile.Reader(inStream,
-				1000,
-				0,
-				100000,
-				new Configuration());
-
-		IntWritable intWritable = new IntWritable();
-		Text txt = new Text();
-
-		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
-			reader.next(intWritable, txt);
-			Assert.assertEquals(i, intWritable.get());
-			Assert.assertEquals("message #" + i, txt.toString());
-		}
-
-		reader.close();
-		inStream.close();
-
-		inStream = dfs.open(new Path(outPath + "/part-1-0"));
-
-		reader = new SequenceFile.Reader(inStream,
-				1000,
-				0,
-				100000,
-				new Configuration());
-
-		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
-			reader.next(intWritable, txt);
-			Assert.assertEquals(i, intWritable.get());
-			Assert.assertEquals("message #" + i, txt.toString());
-		}
-
-		reader.close();
-		inStream.close();
-	}
-	
-	
-	/**
-	 * This tests {@link AvroKeyValueSinkWriter}
-	 * with non-rolling output and without compression.
-	 */
-	@Test
-	public void testNonRollingAvroKeyValueWithoutCompressionWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/avro-kv-no-comp-non-rolling-out";
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
-				.broadcast()
-				.filter(new OddEvenFilter());
-
-
-		Map<String, String> properties = new HashMap<>();
-		Schema keySchema = Schema.create(Type.INT);
-		Schema valueSchema = Schema.create(Type.STRING);
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema.toString());
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema.toString());
-		RollingSink<Tuple2<Integer, String>> sink = new RollingSink<Tuple2<Integer, String>>(outPath)
-				.setWriter(new AvroKeyValueSinkWriter<Integer, String>(properties))
-				.setBucketer(new NonRollingBucketer())
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		source.addSink(sink);
-
-		env.execute("RollingSink Avro KeyValue Writer Test");
-
-		GenericData.setStringType(valueSchema, StringType.String);
-		Schema elementSchema = AvroKeyValue.getSchema(keySchema, valueSchema);
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
-		SpecificDatumReader<GenericRecord> elementReader = new SpecificDatumReader<GenericRecord>(elementSchema);
-		DataFileStream<GenericRecord> dataFileStream = new DataFileStream<GenericRecord>(inStream, elementReader);
-		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
-			AvroKeyValue<Integer, String> wrappedEntry = new AvroKeyValue<Integer, String>(dataFileStream.next());
-			int key = wrappedEntry.getKey().intValue();
-			Assert.assertEquals(i, key);
-			String value = wrappedEntry.getValue();
-			Assert.assertEquals("message #" + i, value);
-		}
-
-		dataFileStream.close();
-		inStream.close();
-
-		inStream = dfs.open(new Path(outPath + "/part-1-0"));
-		dataFileStream = new DataFileStream<GenericRecord>(inStream, elementReader);
-
-		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
-			AvroKeyValue<Integer, String> wrappedEntry = new AvroKeyValue<Integer, String>(dataFileStream.next());
-			int key = wrappedEntry.getKey().intValue();
-			Assert.assertEquals(i, key);
-			String value = wrappedEntry.getValue();
-			Assert.assertEquals("message #" + i, value);
-		}
-
-		dataFileStream.close();
-		inStream.close();
-	}
-	
-	/**
-	 * This tests {@link AvroKeyValueSinkWriter}
-	 * with non-rolling output and with compression.
-	 */
-	@Test
-	public void testNonRollingAvroKeyValueWithCompressionWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/avro-kv-no-comp-non-rolling-out";
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
-				.broadcast()
-				.filter(new OddEvenFilter());
-
-
-		Map<String, String> properties = new HashMap<>();
-		Schema keySchema = Schema.create(Type.INT);
-		Schema valueSchema = Schema.create(Type.STRING);
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema.toString());
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema.toString());
-		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS, String.valueOf(true));
-		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS_CODEC, DataFileConstants.SNAPPY_CODEC);
-		RollingSink<Tuple2<Integer, String>> sink = new RollingSink<Tuple2<Integer, String>>(outPath)
-				.setWriter(new AvroKeyValueSinkWriter<Integer, String>(properties))
-				.setBucketer(new NonRollingBucketer())
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		source.addSink(sink);
-
-		env.execute("RollingSink Avro KeyValue Writer Test");
-
-		GenericData.setStringType(valueSchema, StringType.String);
-		Schema elementSchema = AvroKeyValue.getSchema(keySchema, valueSchema);
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
-		SpecificDatumReader<GenericRecord> elementReader = new SpecificDatumReader<GenericRecord>(elementSchema);
-		DataFileStream<GenericRecord> dataFileStream = new DataFileStream<GenericRecord>(inStream, elementReader);
-		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
-			AvroKeyValue<Integer, String> wrappedEntry = new AvroKeyValue<Integer, String>(dataFileStream.next());
-			int key = wrappedEntry.getKey().intValue();
-			Assert.assertEquals(i, key);
-			String value = wrappedEntry.getValue();
-			Assert.assertEquals("message #" + i, value);
-		}
-
-		dataFileStream.close();
-		inStream.close();
-
-		inStream = dfs.open(new Path(outPath + "/part-1-0"));
-		dataFileStream = new DataFileStream<GenericRecord>(inStream, elementReader);
-
-		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
-			AvroKeyValue<Integer, String> wrappedEntry = new AvroKeyValue<Integer, String>(dataFileStream.next());
-			int key = wrappedEntry.getKey().intValue();
-			Assert.assertEquals(i, key);
-			String value = wrappedEntry.getValue();
-			Assert.assertEquals("message #" + i, value);
-		}
-
-		dataFileStream.close();
-		inStream.close();
-	}
-
-
-	/**
-	 * This tests user defined hdfs configuration
-	 * @throws Exception
-     */
-	@Test
-	public void testUserDefinedConfiguration() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/string-non-rolling-with-config";
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
-			.broadcast()
-			.filter(new OddEvenFilter());
-
-		Configuration conf = new Configuration();
-		conf.set("io.file.buffer.size", "40960");
-		RollingSink<String> sink = new RollingSink<String>(outPath)
-			.setFSConfig(conf)
-			.setWriter(new StreamWriterWithConfigCheck<String>("io.file.buffer.size", "40960"))
-			.setBucketer(new NonRollingBucketer())
-			.setPartPrefix("part")
-			.setPendingPrefix("")
-			.setPendingSuffix("");
-
-		source
-			.map(new MapFunction<Tuple2<Integer,String>, String>() {
-				private static final long serialVersionUID = 1L;
-				@Override
-				public String map(Tuple2<Integer, String> value) throws Exception {
-					return value.f1;
-				}
-			})
-			.addSink(sink);
-
-		env.execute("RollingSink with configuration Test");
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
-
-		BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
-			String line = br.readLine();
-			Assert.assertEquals("message #" + i, line);
-		}
-
-		inStream.close();
-
-		inStream = dfs.open(new Path(outPath + "/part-1-0"));
-
-		br = new BufferedReader(new InputStreamReader(inStream));
-
-		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
-			String line = br.readLine();
-			Assert.assertEquals("message #" + i, line);
-		}
-
-		inStream.close();
-	}
-
-	// we use this to synchronize the clock changes to elements being processed
-	final static MultiShotLatch latch1 = new MultiShotLatch();
-	final static MultiShotLatch latch2 = new MultiShotLatch();
-
-	/**
-	 * This uses {@link org.apache.flink.streaming.connectors.fs.DateTimeBucketer} to
-	 * produce rolling files. The clock of DateTimeBucketer is set to
-	 * {@link ModifyableClock} to keep the time in lockstep with the processing of elements using
-	 * latches.
-	 */
-	@Test
-	public void testDateTimeRollingStringWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/rolling-out";
-		DateTimeBucketer.setClock(new ModifyableClock());
-		ModifyableClock.setCurrentTime(0);
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new WaitingTestSourceFunction(
-				NUM_ELEMENTS))
-				.broadcast();
-
-		// the parallel flatMap is chained to the sink, so when it has seen 5 elements it can
-		// fire the latch
-		DataStream<String> mapped = source
-				.flatMap(new RichFlatMapFunction<Tuple2<Integer, String>, String>() {
-					private static final long serialVersionUID = 1L;
-
-					int count = 0;
-					@Override
-					public void flatMap(Tuple2<Integer, String> value,
-							Collector<String> out) throws Exception {
-						out.collect(value.f1);
-						count++;
-						if (count >= 5) {
-							if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
-								latch1.trigger();
-							} else {
-								latch2.trigger();
-							}
-							count = 0;
-						}
-					}
-
-				});
-
-		RollingSink<String> sink = new RollingSink<String>(outPath)
-				.setBucketer(new DateTimeBucketer("ss"))
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		mapped.addSink(sink);
-
-		env.execute("RollingSink String Write Test");
-
-		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(outPath), true);
-
-		// we should have 8 rolling files, 4 time intervals and parallelism of 2
-		int numFiles = 0;
-		while (files.hasNext()) {
-			LocatedFileStatus file = files.next();
-			numFiles++;
-			if (file.getPath().toString().contains("rolling-out/00")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 0; i < 5; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else if (file.getPath().toString().contains("rolling-out/05")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 5; i < 10; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else if (file.getPath().toString().contains("rolling-out/10")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 10; i < 15; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else if (file.getPath().toString().contains("rolling-out/15")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 15; i < 20; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else {
-				Assert.fail("File " + file + " does not match any expected roll pattern.");
-			}
-		}
-
-		Assert.assertEquals(8, numFiles);
-	}
-
-	private static final String PART_PREFIX = "part";
-	private static final String PENDING_SUFFIX = ".pending";
-	private static final String IN_PROGRESS_SUFFIX = ".in-progress";
-	private static final String VALID_LENGTH_SUFFIX = ".valid";
-
-	@Test
-	public void testBucketStateTransitions() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness = createRescalingTestSink(outDir, 1, 0);
-		testHarness.setup();
-		testHarness.open();
-
-		testHarness.setProcessingTime(0L);
-
-		// we have a bucket size of 5 bytes, so each record will get its own bucket,
-		// i.e. the bucket should roll after every record.
-
-		testHarness.processElement(new StreamRecord<>("test1", 1L));
-		testHarness.processElement(new StreamRecord<>("test2", 1L));
-		checkFs(outDir, 1, 1 ,0, 0);
-
-		testHarness.processElement(new StreamRecord<>("test3", 1L));
-		checkFs(outDir, 1, 2, 0, 0);
-
-		testHarness.snapshot(0, 0);
-		checkFs(outDir, 1, 2, 0, 0);
-
-		testHarness.notifyOfCompletedCheckpoint(0);
-		checkFs(outDir, 1, 0, 2, 0);
-
-		OperatorStateHandles snapshot = testHarness.snapshot(1, 0);
-
-		testHarness.close();
-		checkFs(outDir, 0, 1, 2, 0);
-
-		testHarness = createRescalingTestSink(outDir, 1, 0);
-		testHarness.setup();
-		testHarness.initializeState(snapshot);
-		testHarness.open();
-		checkFs(outDir, 0, 0, 3, 1);
-
-		snapshot = testHarness.snapshot(2, 0);
-
-		testHarness.processElement(new StreamRecord<>("test4", 10));
-		checkFs(outDir, 1, 0, 3, 1);
-
-		testHarness = createRescalingTestSink(outDir, 1, 0);
-		testHarness.setup();
-		testHarness.initializeState(snapshot);
-		testHarness.open();
-
-		// the in-progress file remains as we do not clean up now
-		checkFs(outDir, 1, 0, 3, 1);
-
-		testHarness.close();
-
-		// at close it is not moved to final because it is not part
-		// of the current task's state, it was just a not cleaned up leftover.
-		checkFs(outDir, 1, 0, 3, 1);
-	}
-
-	@Test
-	public void testScalingDown() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 3, 0);
-		testHarness1.setup();
-		testHarness1.open();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 3, 1);
-		testHarness2.setup();
-		testHarness2.open();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness3 = createRescalingTestSink(outDir, 3, 2);
-		testHarness3.setup();
-		testHarness3.open();
-
-		testHarness1.processElement(new StreamRecord<>("test1", 0L));
-		checkFs(outDir, 1, 0, 0, 0);
-
-		testHarness2.processElement(new StreamRecord<>("test2", 0L));
-		testHarness2.processElement(new StreamRecord<>("test3", 0L));
-		testHarness2.processElement(new StreamRecord<>("test4", 0L));
-		testHarness2.processElement(new StreamRecord<>("test5", 0L));
-		testHarness2.processElement(new StreamRecord<>("test6", 0L));
-		checkFs(outDir, 2, 4, 0, 0);
-
-		testHarness3.processElement(new StreamRecord<>("test7", 0L));
-		testHarness3.processElement(new StreamRecord<>("test8", 0L));
-		checkFs(outDir, 3, 5, 0, 0);
-
-		// intentionally we snapshot them in a not ascending order so that the states are shuffled
-		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
-			testHarness3.snapshot(0, 0),
-			testHarness1.snapshot(0, 0),
-			testHarness2.snapshot(0, 0)
-		);
-
-		// with the above state reshuffling, we expect testHarness4 to take the
-		// state of the previous testHarness3 and testHarness1 while testHarness5
-		// will take that of the previous testHarness1
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness4 = createRescalingTestSink(outDir, 2, 0);
-		testHarness4.setup();
-		testHarness4.initializeState(mergedSnapshot);
-		testHarness4.open();
-
-		// we do not have a length file for part-2-0 because bucket part-2-0
-		// was not "in-progress", but "pending" (its full content is valid).
-		checkFs(outDir, 1, 4, 3, 2);
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness5 = createRescalingTestSink(outDir, 2, 1);
-		testHarness5.setup();
-		testHarness5.initializeState(mergedSnapshot);
-		testHarness5.open();
-
-		checkFs(outDir, 0, 0, 8, 3);
-	}
-
-	@Test
-	public void testScalingUp() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 2, 0);
-		testHarness1.setup();
-		testHarness1.open();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 2, 0);
-		testHarness2.setup();
-		testHarness2.open();
-
-		testHarness1.processElement(new StreamRecord<>("test1", 0L));
-		testHarness1.processElement(new StreamRecord<>("test2", 0L));
-
-		checkFs(outDir, 1, 1, 0, 0);
-
-		testHarness2.processElement(new StreamRecord<>("test3", 0L));
-		testHarness2.processElement(new StreamRecord<>("test4", 0L));
-		testHarness2.processElement(new StreamRecord<>("test5", 0L));
-
-		checkFs(outDir, 2, 3, 0, 0);
-
-		// intentionally we snapshot them in the reverse order so that the states are shuffled
-		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
-			testHarness2.snapshot(0, 0),
-			testHarness1.snapshot(0, 0)
-		);
-
-		testHarness1 = createRescalingTestSink(outDir, 3, 0);
-		testHarness1.setup();
-		testHarness1.initializeState(mergedSnapshot);
-		testHarness1.open();
-
-		checkFs(outDir, 1, 1, 3, 1);
-
-		testHarness2 = createRescalingTestSink(outDir, 3, 1);
-		testHarness2.setup();
-		testHarness2.initializeState(mergedSnapshot);
-		testHarness2.open();
-
-		checkFs(outDir, 0, 0, 5, 2);
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness3 = createRescalingTestSink(outDir, 3, 2);
-		testHarness3.setup();
-		testHarness3.initializeState(mergedSnapshot);
-		testHarness3.open();
-
-		checkFs(outDir, 0, 0, 5, 2);
-
-		testHarness1.processElement(new StreamRecord<>("test6", 0));
-		testHarness2.processElement(new StreamRecord<>("test6", 0));
-		testHarness3.processElement(new StreamRecord<>("test6", 0));
-
-		// 3 for the different tasks
-		checkFs(outDir, 3, 0, 5, 2);
-
-		testHarness1.snapshot(1, 0);
-		testHarness2.snapshot(1, 0);
-		testHarness3.snapshot(1, 0);
-
-		testHarness1.close();
-		testHarness2.close();
-		testHarness3.close();
-
-		checkFs(outDir, 0, 3, 5, 2);
-	}
-
-	private OneInputStreamOperatorTestHarness<String, Object> createRescalingTestSink(
-		File outDir, int totalParallelism, int taskIdx) throws Exception {
-
-		RollingSink<String> sink = new RollingSink<String>(outDir.getAbsolutePath())
-			.setWriter(new StringWriter<String>())
-			.setBatchSize(5)
-			.setPartPrefix(PART_PREFIX)
-			.setInProgressPrefix("")
-			.setPendingPrefix("")
-			.setValidLengthPrefix("")
-			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
-			.setPendingSuffix(PENDING_SUFFIX)
-			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
-
-		return createTestSink(sink, totalParallelism, taskIdx);
-	}
-
-	private <T> OneInputStreamOperatorTestHarness<T, Object> createTestSink(
-		RollingSink<T> sink, int totalParallelism, int taskIdx) throws Exception {
-		return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), 10, totalParallelism, taskIdx);
-	}
-
-	private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
-		int inProg = 0;
-		int pend = 0;
-		int compl = 0;
-		int val = 0;
-
-		for (File file: FileUtils.listFiles(outDir, null, true)) {
-			if (file.getAbsolutePath().endsWith("crc")) {
-				continue;
-			}
-			String path = file.getPath();
-			if (path.endsWith(IN_PROGRESS_SUFFIX)) {
-				inProg++;
-			} else if (path.endsWith(PENDING_SUFFIX)) {
-				pend++;
-			} else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
-				val++;
-			} else if (path.contains(PART_PREFIX)) {
-				compl++;
-			}
-		}
-
-		Assert.assertEquals(inprogress, inProg);
-		Assert.assertEquals(pending, pend);
-		Assert.assertEquals(completed, compl);
-		Assert.assertEquals(valid, val);
-	}
-
-	private static class TestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		private volatile boolean running = true;
-
-		private final int numElements;
-
-		public TestSourceFunction(int numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
-			for (int i = 0; i < numElements && running; i++) {
-				ctx.collect(Tuple2.of(i, "message #" + i));
-			}
-		}
-
-		@Override
-		public void cancel() {
-			running = false;
-		}
-	}
-
-	/**
-	 * This waits on the two multi-shot latches. The latches are triggered in a parallel
-	 * flatMap inside the test topology.
-	 */
-	private static class WaitingTestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		private volatile boolean running = true;
-
-		private final int numElements;
-
-		public WaitingTestSourceFunction(int numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
-			for (int i = 0; i < numElements && running; i++) {
-				if (i % 5 == 0 && i > 0) {
-					// update the clock after "five seconds", so we get 20 seconds in total
-					// with 5 elements in each time window
-					latch1.await();
-					latch2.await();
-					ModifyableClock.setCurrentTime(i * 1000);
-				}
-				ctx.collect(Tuple2.of(i, "message #" + i));
-			}
-		}
-
-		@Override
-		public void cancel() {
-			running = false;
-		}
-	}
-
-
-	private static class StreamWriterWithConfigCheck<T> extends StringWriter<T> {
-		private String key;
-		private String expect;
-		public StreamWriterWithConfigCheck(String key, String expect) {
-			this.key = key;
-			this.expect = expect;
-		}
-
-		@Override
-		public void open(FileSystem fs, Path path) throws IOException {
-			super.open(fs, path);
-			Assert.assertEquals(expect, fs.getConf().get(key));
-		}
-
-		@Override
-		public Writer<T> duplicate() {
-			return new StreamWriterWithConfigCheck<>(key, expect);
-		}
-	}
-
-	public static class OddEvenFilter extends RichFilterFunction<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Tuple2<Integer, String> value) throws Exception {
-			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
-				return value.f0 % 2 == 0;
-			} else {
-				return value.f0 % 2 == 1;
-			}
-		}
-	}
-
-	public static class ModifyableClock implements Clock {
-
-		private static volatile long currentTime = 0;
-
-		public static void setCurrentTime(long currentTime) {
-			ModifyableClock.currentTime = currentTime;
-		}
-
-		@Override
-		public long currentTimeMillis() {
-			return currentTime;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
deleted file mode 100644
index eb12d07..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
+++ /dev/null
@@ -1,252 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.fs;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.runtime.security.SecurityUtils;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.test.util.SecureTestEnvironment;
-import org.apache.flink.test.util.TestingSecurityContext;
-import org.apache.flink.test.util.TestBaseUtils;
-import org.apache.flink.util.NetUtils;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.VersionInfo;
-import org.junit.AfterClass;
-import org.junit.Assume;
-import org.junit.BeforeClass;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTP_POLICY_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
-
-/**
- * Tests for running {@link RollingSinkSecuredITCase} which is an extension of {@link RollingSink} in secure environment
- * Note: only executed for Hadoop version > 3.x.x
- */
-public class RollingSinkSecuredITCase extends RollingSinkITCase {
-
-	protected static final Logger LOG = LoggerFactory.getLogger(RollingSinkSecuredITCase.class);
-
-	/**
-	 * Skips all tests if the Hadoop version doesn't match.
-	 * We can't run this test class until HDFS-9213 is fixed which allows a secure DataNode
-	 * to bind to non-privileged ports for testing.
-	 * For now, we skip this test class until Hadoop version 3.x.x.
-	 */
-	private static void skipIfHadoopVersionIsNotAppropriate() {
-		// Skips all tests if the Hadoop version doesn't match
-		String hadoopVersionString = VersionInfo.getVersion();
-		String[] split = hadoopVersionString.split("\\.");
-		if (split.length != 3) {
-			throw new IllegalStateException("Hadoop version was not of format 'X.X.X': " + hadoopVersionString);
-		}
-		Assume.assumeTrue(
-			// check whether we're running Hadoop version >= 3.x.x
-			Integer.parseInt(split[0]) >= 3
-		);
-	}
-
-	/*
-	 * override super class static methods to avoid creating MiniDFS and MiniFlink with wrong configurations
-	 * and out-of-order sequence for secure cluster
-	 */
-	@BeforeClass
-	public static void setup() throws Exception {}
-
-	@AfterClass
-	public static void teardown() throws Exception {}
-
-	@BeforeClass
-	public static void createHDFS() throws IOException {}
-
-	@AfterClass
-	public static void destroyHDFS() {}
-
-	@BeforeClass
-	public static void startSecureCluster() throws Exception {
-
-		skipIfHadoopVersionIsNotAppropriate();
-
-		LOG.info("starting secure cluster environment for testing");
-
-		dataDir = tempFolder.newFolder();
-
-		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
-
-		SecureTestEnvironment.prepare(tempFolder);
-
-		populateSecureConfigurations();
-
-		Configuration flinkConfig = new Configuration();
-		flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY,
-				SecureTestEnvironment.getTestKeytab());
-		flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY,
-				SecureTestEnvironment.getHadoopServicePrincipal());
-
-		SecurityUtils.SecurityConfiguration ctx = new SecurityUtils.SecurityConfiguration(flinkConfig);
-		ctx.setHadoopConfiguration(conf);
-		try {
-			TestingSecurityContext.install(ctx, SecureTestEnvironment.getClientSecurityConfigurationMap());
-		} catch (Exception e) {
-			throw new RuntimeException("Exception occurred while setting up secure test context. Reason: {}", e);
-		}
-
-		File hdfsSiteXML = new File(dataDir.getAbsolutePath() + "/hdfs-site.xml");
-
-		FileWriter writer = new FileWriter(hdfsSiteXML);
-		conf.writeXml(writer);
-		writer.flush();
-		writer.close();
-
-		Map<String, String> map = new HashMap<String, String>(System.getenv());
-		map.put("HADOOP_CONF_DIR", hdfsSiteXML.getParentFile().getAbsolutePath());
-		TestBaseUtils.setEnv(map);
-
-
-		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-		builder.checkDataNodeAddrConfig(true);
-		builder.checkDataNodeHostConfig(true);
-		hdfsCluster = builder.build();
-
-		dfs = hdfsCluster.getFileSystem();
-
-		hdfsURI = "hdfs://"
-				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
-				+ "/";
-
-		startSecureFlinkClusterWithRecoveryModeEnabled();
-	}
-
-	@AfterClass
-	public static void teardownSecureCluster() throws Exception {
-		LOG.info("tearing down secure cluster environment");
-
-		TestStreamEnvironment.unsetAsContext();
-		stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT);
-
-		if (hdfsCluster != null) {
-			hdfsCluster.shutdown();
-		}
-
-		SecureTestEnvironment.cleanup();
-	}
-
-	private static void populateSecureConfigurations() {
-
-		String dataTransferProtection = "authentication";
-
-		SecurityUtil.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
-		conf.set(DFS_NAMENODE_USER_NAME_KEY, SecureTestEnvironment.getHadoopServicePrincipal());
-		conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, SecureTestEnvironment.getTestKeytab());
-		conf.set(DFS_DATANODE_USER_NAME_KEY, SecureTestEnvironment.getHadoopServicePrincipal());
-		conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, SecureTestEnvironment.getTestKeytab());
-		conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, SecureTestEnvironment.getHadoopServicePrincipal());
-
-		conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
-
-		conf.set("dfs.data.transfer.protection", dataTransferProtection);
-
-		conf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTP_ONLY.name());
-
-		conf.set(DFS_ENCRYPT_DATA_TRANSFER_KEY, "false");
-
-		conf.setInt("dfs.datanode.socket.write.timeout", 0);
-
-		/*
-		 * We ae setting the port number to privileged port - see HDFS-9213
-		 * This requires the user to have root privilege to bind to the port
-		 * Use below command (ubuntu) to set privilege to java process for the
-		 * bind() to work if the java process is not running as root.
-		 * setcap 'cap_net_bind_service=+ep' /path/to/java
-		 */
-		conf.set(DFS_DATANODE_ADDRESS_KEY, "localhost:1002");
-		conf.set(DFS_DATANODE_HOST_NAME_KEY, "localhost");
-		conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "localhost:1003");
-	}
-
-	private static void startSecureFlinkClusterWithRecoveryModeEnabled() {
-		try {
-			LOG.info("Starting Flink and ZK in secure mode");
-
-			dfs.mkdirs(new Path("/flink/checkpoints"));
-			dfs.mkdirs(new Path("/flink/recovery"));
-
-			org.apache.flink.configuration.Configuration config = new org.apache.flink.configuration.Configuration();
-
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, DEFAULT_PARALLELISM);
-			config.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false);
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3);
-			config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-			config.setString(ConfigConstants.STATE_BACKEND, "filesystem");
-			config.setString(ConfigConstants.ZOOKEEPER_CHECKPOINTS_PATH, hdfsURI + "/flink/checkpoints");
-			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, hdfsURI + "/flink/recovery");
-			config.setString("state.backend.fs.checkpointdir", hdfsURI + "/flink/checkpoints");
-
-			SecureTestEnvironment.populateFlinkSecureConfigurations(config);
-
-			cluster = TestBaseUtils.startCluster(config, false);
-			TestStreamEnvironment.setAsContext(cluster, DEFAULT_PARALLELISM);
-
-		} catch (Exception e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-	/* For secure cluster testing, it is enough to run only one test and override below test methods
-	 * to keep the overall build time minimal
-	 */
-	@Override
-	public void testNonRollingSequenceFileWithoutCompressionWriter() throws Exception {}
-
-	@Override
-	public void testNonRollingSequenceFileWithCompressionWriter() throws Exception {}
-
-	@Override
-	public void testNonRollingAvroKeyValueWithoutCompressionWriter() throws Exception {}
-
-	@Override
-	public void testNonRollingAvroKeyValueWithCompressionWriter() throws Exception {}
-
-	@Override
-	public void testDateTimeRollingStringWriter() throws Exception {}
-
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
deleted file mode 100644
index d671874..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
+++ /dev/null
@@ -1,867 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs.bucketing;
-
-import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileConstants;
-import org.apache.avro.file.DataFileStream;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.commons.io.FileUtils;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.fs.AvroKeyValueSinkWriter;
-import org.apache.flink.streaming.connectors.fs.Clock;
-import org.apache.flink.streaming.connectors.fs.SequenceFileWriter;
-import org.apache.flink.streaming.connectors.fs.StringWriter;
-import org.apache.flink.streaming.connectors.fs.Writer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.util.NetUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.Assert;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.HashMap;
-import java.util.Map;
-
-public class BucketingSinkTest {
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	private static MiniDFSCluster hdfsCluster;
-	private static org.apache.hadoop.fs.FileSystem dfs;
-	private static String hdfsURI;
-
-	private static final String PART_PREFIX = "part";
-	private static final String PENDING_SUFFIX = ".pending";
-	private static final String IN_PROGRESS_SUFFIX = ".in-progress";
-	private static final String VALID_LENGTH_SUFFIX = ".valid";
-
-	private OneInputStreamOperatorTestHarness<String, Object> createRescalingTestSink(
-		File outDir, int totalParallelism, int taskIdx, long inactivityInterval) throws Exception {
-
-		BucketingSink<String> sink = new BucketingSink<String>(outDir.getAbsolutePath())
-			.setBucketer(new Bucketer<String>() {
-				private static final long serialVersionUID = 1L;
-
-				@Override
-				public Path getBucketPath(Clock clock, Path basePath, String element) {
-					return new Path(basePath, element);
-				}
-			})
-			.setWriter(new StringWriter<String>())
-			.setInactiveBucketCheckInterval(inactivityInterval)
-			.setInactiveBucketThreshold(inactivityInterval)
-			.setPartPrefix(PART_PREFIX)
-			.setInProgressPrefix("")
-			.setPendingPrefix("")
-			.setValidLengthPrefix("")
-			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
-			.setPendingSuffix(PENDING_SUFFIX)
-			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
-
-		return createTestSink(sink, totalParallelism, taskIdx);
-	}
-
-	private OneInputStreamOperatorTestHarness<String, Object> createTestSink(File dataDir, int totalParallelism, int taskIdx) throws Exception {
-		BucketingSink<String> sink = new BucketingSink<String>(dataDir.getAbsolutePath())
-			.setBucketer(new Bucketer<String>() {
-				private static final long serialVersionUID = 1L;
-
-				@Override
-				public Path getBucketPath(Clock clock, Path basePath, String element) {
-					return new Path(basePath, element);
-				}
-			})
-			.setWriter(new StringWriter<String>())
-			.setPartPrefix(PART_PREFIX)
-			.setPendingPrefix("")
-			.setInactiveBucketCheckInterval(5*60*1000L)
-			.setInactiveBucketThreshold(5*60*1000L)
-			.setPendingSuffix(PENDING_SUFFIX)
-			.setInProgressSuffix(IN_PROGRESS_SUFFIX);
-
-		return createTestSink(sink, totalParallelism, taskIdx);
-	}
-
-	private <T> OneInputStreamOperatorTestHarness<T, Object> createTestSink(
-			BucketingSink<T> sink, int totalParallelism, int taskIdx) throws Exception {
-		return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), 10, totalParallelism, taskIdx);
-	}
-
-	@BeforeClass
-	public static void createHDFS() throws IOException {
-		Configuration conf = new Configuration();
-
-		File dataDir = tempFolder.newFolder();
-
-		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
-		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-		hdfsCluster = builder.build();
-
-		dfs = hdfsCluster.getFileSystem();
-
-		hdfsURI = "hdfs://"
-			+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
-			+ "/";
-	}
-
-	@AfterClass
-	public static void destroyHDFS() {
-		hdfsCluster.shutdown();
-	}
-
-	@Test
-	public void testInactivityPeriodWithLateNotify() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness = createRescalingTestSink(outDir, 1, 0, 100);
-		testHarness.setup();
-		testHarness.open();
-
-		testHarness.setProcessingTime(0L);
-
-		testHarness.processElement(new StreamRecord<>("test1", 1L));
-		testHarness.processElement(new StreamRecord<>("test2", 1L));
-		checkFs(outDir, 2, 0 ,0, 0);
-
-		testHarness.setProcessingTime(101L);	// put some in pending
-		checkFs(outDir, 0, 2, 0, 0);
-
-		testHarness.snapshot(0, 0);				// put them in pending for 0
-		checkFs(outDir, 0, 2, 0, 0);
-
-		testHarness.processElement(new StreamRecord<>("test3", 1L));
-		testHarness.processElement(new StreamRecord<>("test4", 1L));
-
-		testHarness.setProcessingTime(202L);	// put some in pending
-
-		testHarness.snapshot(1, 0);				// put them in pending for 1
-		checkFs(outDir, 0, 4, 0, 0);
-
-		testHarness.notifyOfCompletedCheckpoint(0);	// put the pending for 0 to the "committed" state
-		checkFs(outDir, 0, 2, 2, 0);
-
-		testHarness.notifyOfCompletedCheckpoint(1); // put the pending for 1 to the "committed" state
-		checkFs(outDir, 0, 0, 4, 0);
-	}
-
-	@Test
-	public void testBucketStateTransitions() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness = createRescalingTestSink(outDir, 1, 0, 100);
-		testHarness.setup();
-		testHarness.open();
-
-		testHarness.setProcessingTime(0L);
-
-		testHarness.processElement(new StreamRecord<>("test1", 1L));
-		testHarness.processElement(new StreamRecord<>("test2", 1L));
-		checkFs(outDir, 2, 0 ,0, 0);
-
-		// this is to check the inactivity threshold
-		testHarness.setProcessingTime(101L);
-		checkFs(outDir, 0, 2, 0, 0);
-
-		testHarness.processElement(new StreamRecord<>("test3", 1L));
-		checkFs(outDir, 1, 2, 0, 0);
-
-		testHarness.snapshot(0, 0);
-		checkFs(outDir, 1, 2, 0, 0);
-
-		testHarness.notifyOfCompletedCheckpoint(0);
-		checkFs(outDir, 1, 0, 2, 0);
-
-		OperatorStateHandles snapshot = testHarness.snapshot(1, 0);
-
-		testHarness.close();
-		checkFs(outDir, 0, 1, 2, 0);
-
-		testHarness = createRescalingTestSink(outDir, 1, 0, 100);
-		testHarness.setup();
-		testHarness.initializeState(snapshot);
-		testHarness.open();
-		checkFs(outDir, 0, 0, 3, 1);
-
-		snapshot = testHarness.snapshot(2, 0);
-
-		testHarness.processElement(new StreamRecord<>("test4", 10));
-		checkFs(outDir, 1, 0, 3, 1);
-
-		testHarness = createRescalingTestSink(outDir, 1, 0, 100);
-		testHarness.setup();
-		testHarness.initializeState(snapshot);
-		testHarness.open();
-
-		// the in-progress file remains as we do not clean up now
-		checkFs(outDir, 1, 0, 3, 1);
-
-		testHarness.close();
-
-		// at close it is not moved to final because it is not part
-		// of the current task's state, it was just a not cleaned up leftover.
-		checkFs(outDir, 1, 0, 3, 1);
-	}
-
-	@Test
-	public void testSameParallelismWithShufflingStates() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 2, 0, 100);
-		testHarness1.setup();
-		testHarness1.open();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 2, 1, 100);
-		testHarness2.setup();
-		testHarness2.open();
-
-		testHarness1.processElement(new StreamRecord<>("test1", 0L));
-		checkFs(outDir, 1, 0, 0, 0);
-
-		testHarness2.processElement(new StreamRecord<>("test2", 0L));
-		checkFs(outDir, 2, 0, 0, 0);
-
-		// intentionally we snapshot them in the reverse order so that the states are shuffled
-		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
-			testHarness2.snapshot(0, 0),
-			testHarness1.snapshot(0, 0)
-		);
-
-		checkFs(outDir, 2, 0, 0, 0);
-
-		// this will not be included in any checkpoint so it can be cleaned up (although we do not)
-		testHarness2.processElement(new StreamRecord<>("test3", 0L));
-		checkFs(outDir, 3, 0, 0, 0);
-
-		testHarness1 = createRescalingTestSink(outDir, 2, 0, 100);
-		testHarness1.setup();
-		testHarness1.initializeState(mergedSnapshot);
-		testHarness1.open();
-
-		// the one in-progress will be the one assigned to the next instance,
-		// the other is the test3 which is just not cleaned up
-		checkFs(outDir, 2, 0, 1, 1);
-
-		testHarness2 = createRescalingTestSink(outDir, 2, 1, 100);
-		testHarness2.setup();
-		testHarness2.initializeState(mergedSnapshot);
-		testHarness2.open();
-
-		checkFs(outDir, 1, 0, 2, 2);
-
-		testHarness1.close();
-		testHarness2.close();
-
-		// the 1 in-progress can be discarded.
-		checkFs(outDir, 1, 0, 2, 2);
-	}
-
-	@Test
-	public void testScalingDown() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 3, 0, 100);
-		testHarness1.setup();
-		testHarness1.open();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 3, 1, 100);
-		testHarness2.setup();
-		testHarness2.open();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness3 = createRescalingTestSink(outDir, 3, 2, 100);
-		testHarness3.setup();
-		testHarness3.open();
-
-		testHarness1.processElement(new StreamRecord<>("test1", 0L));
-		checkFs(outDir, 1, 0, 0, 0);
-
-		testHarness2.processElement(new StreamRecord<>("test2", 0L));
-		checkFs(outDir, 2, 0, 0, 0);
-
-		testHarness3.processElement(new StreamRecord<>("test3", 0L));
-		testHarness3.processElement(new StreamRecord<>("test4", 0L));
-		checkFs(outDir, 4, 0, 0, 0);
-
-		// intentionally we snapshot them in the reverse order so that the states are shuffled
-		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
-			testHarness3.snapshot(0, 0),
-			testHarness1.snapshot(0, 0),
-			testHarness2.snapshot(0, 0)
-		);
-
-		testHarness1 = createRescalingTestSink(outDir, 2, 0, 100);
-		testHarness1.setup();
-		testHarness1.initializeState(mergedSnapshot);
-		testHarness1.open();
-
-		checkFs(outDir, 1, 0, 3, 3);
-
-		testHarness2 = createRescalingTestSink(outDir, 2, 1, 100);
-		testHarness2.setup();
-		testHarness2.initializeState(mergedSnapshot);
-		testHarness2.open();
-
-		checkFs(outDir, 0, 0, 4, 4);
-	}
-
-	@Test
-	public void testScalingUp() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 2, 0, 100);
-		testHarness1.setup();
-		testHarness1.open();
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 2, 0, 100);
-		testHarness2.setup();
-		testHarness2.open();
-
-		testHarness1.processElement(new StreamRecord<>("test1", 1L));
-		testHarness1.processElement(new StreamRecord<>("test2", 1L));
-
-		checkFs(outDir, 2, 0, 0, 0);
-
-		testHarness2.processElement(new StreamRecord<>("test3", 1L));
-		testHarness2.processElement(new StreamRecord<>("test4", 1L));
-		testHarness2.processElement(new StreamRecord<>("test5", 1L));
-
-		checkFs(outDir, 5, 0, 0, 0);
-
-		// intentionally we snapshot them in the reverse order so that the states are shuffled
-		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
-			testHarness2.snapshot(0, 0),
-			testHarness1.snapshot(0, 0)
-		);
-
-		testHarness1 = createRescalingTestSink(outDir, 3, 0, 100);
-		testHarness1.setup();
-		testHarness1.initializeState(mergedSnapshot);
-		testHarness1.open();
-
-		checkFs(outDir, 2, 0, 3, 3);
-
-		testHarness2 = createRescalingTestSink(outDir, 3, 1, 100);
-		testHarness2.setup();
-		testHarness2.initializeState(mergedSnapshot);
-		testHarness2.open();
-
-		checkFs(outDir, 0, 0, 5, 5);
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness3 = createRescalingTestSink(outDir, 3, 2, 100);
-		testHarness3.setup();
-		testHarness3.initializeState(mergedSnapshot);
-		testHarness3.open();
-
-		checkFs(outDir, 0, 0, 5, 5);
-
-		testHarness1.processElement(new StreamRecord<>("test6", 0));
-		testHarness2.processElement(new StreamRecord<>("test6", 0));
-		testHarness3.processElement(new StreamRecord<>("test6", 0));
-
-		checkFs(outDir, 3, 0, 5, 5);
-
-		testHarness1.snapshot(1, 0);
-		testHarness2.snapshot(1, 0);
-		testHarness3.snapshot(1, 0);
-
-		testHarness1.close();
-		testHarness2.close();
-		testHarness3.close();
-
-		checkFs(outDir, 0, 3, 5, 5);
-	}
-
-	private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
-		int inProg = 0;
-		int pend = 0;
-		int compl = 0;
-		int val = 0;
-
-		for (File file: FileUtils.listFiles(outDir, null, true)) {
-			if (file.getAbsolutePath().endsWith("crc")) {
-				continue;
-			}
-			String path = file.getPath();
-			if (path.endsWith(IN_PROGRESS_SUFFIX)) {
-				inProg++;
-			} else if (path.endsWith(PENDING_SUFFIX)) {
-				pend++;
-			} else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
-				val++;
-			} else if (path.contains(PART_PREFIX)) {
-				compl++;
-			}
-		}
-
-		Assert.assertEquals(inprogress, inProg);
-		Assert.assertEquals(pending, pend);
-		Assert.assertEquals(completed, compl);
-		Assert.assertEquals(valid, val);
-	}
-
-	/**
-	 * This tests {@link StringWriter} with
-	 * non-bucketing output.
-	 */
-	@Test
-	public void testNonRollingStringWriter() throws Exception {
-		final String outPath = hdfsURI + "/string-non-rolling-out";
-
-		final int numElements = 20;
-
-		BucketingSink<String> sink = new BucketingSink<String>(outPath)
-			.setBucketer(new BasePathBucketer<String>())
-			.setPartPrefix(PART_PREFIX)
-			.setPendingPrefix("")
-			.setPendingSuffix("");
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(sink, 1, 0);
-
-		testHarness.setProcessingTime(0L);
-
-		testHarness.setup();
-		testHarness.open();
-
-		for (int i = 0; i < numElements; i++) {
-			testHarness.processElement(new StreamRecord<>("message #" + Integer.toString(i)));
-		}
-
-		testHarness.close();
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/" + PART_PREFIX + "-0-0"));
-
-		BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-		for (int i = 0; i < numElements; i++) {
-			String line = br.readLine();
-			Assert.assertEquals("message #" + i, line);
-		}
-
-		inStream.close();
-	}
-
-	/**
-	 * This tests {@link SequenceFileWriter}
-	 * with non-rolling output and without compression.
-	 */
-	@Test
-	public void testNonRollingSequenceFileWithoutCompressionWriter() throws Exception {
-		final String outPath = hdfsURI + "/seq-no-comp-non-rolling-out";
-
-		final int numElements = 20;
-
-		BucketingSink<Tuple2<IntWritable, Text>> sink = new BucketingSink<Tuple2<IntWritable, Text>>(outPath)
-			.setWriter(new SequenceFileWriter<IntWritable, Text>())
-			.setBucketer(new BasePathBucketer<Tuple2<IntWritable, Text>>())
-			.setPartPrefix(PART_PREFIX)
-			.setPendingPrefix("")
-			.setPendingSuffix("");
-
-		sink.setInputType(TypeInformation.of(new TypeHint<Tuple2<IntWritable, Text>>(){}), new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Tuple2<IntWritable, Text>, Object> testHarness =
-			createTestSink(sink, 1, 0);
-
-		testHarness.setProcessingTime(0L);
-
-		testHarness.setup();
-		testHarness.open();
-
-		for (int i = 0; i < numElements; i++) {
-			testHarness.processElement(new StreamRecord<>(Tuple2.of(
-				new IntWritable(i),
-				new Text("message #" + Integer.toString(i))
-			)));
-		}
-
-		testHarness.close();
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/" + PART_PREFIX + "-0-0"));
-
-		SequenceFile.Reader reader = new SequenceFile.Reader(inStream, 1000, 0, 100000, new Configuration());
-
-		IntWritable intWritable = new IntWritable();
-		Text txt = new Text();
-
-		for (int i = 0; i < numElements; i++) {
-			reader.next(intWritable, txt);
-			Assert.assertEquals(i, intWritable.get());
-			Assert.assertEquals("message #" + i, txt.toString());
-		}
-
-		reader.close();
-		inStream.close();
-	}
-
-	/**
-	 * This tests {@link AvroKeyValueSinkWriter}
-	 * with non-rolling output and with compression.
-	 */
-	@Test
-	public void testNonRollingAvroKeyValueWithCompressionWriter() throws Exception {
-		final String outPath = hdfsURI + "/avro-kv-no-comp-non-rolling-out";
-
-		final int numElements = 20;
-
-		Map<String, String> properties = new HashMap<>();
-		Schema keySchema = Schema.create(Schema.Type.INT);
-		Schema valueSchema = Schema.create(Schema.Type.STRING);
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema.toString());
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema.toString());
-		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS, String.valueOf(true));
-		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS_CODEC, DataFileConstants.SNAPPY_CODEC);
-
-		BucketingSink<Tuple2<Integer, String>> sink = new BucketingSink<Tuple2<Integer, String>>(outPath)
-			.setWriter(new AvroKeyValueSinkWriter<Integer, String>(properties))
-			.setBucketer(new BasePathBucketer<Tuple2<Integer, String>>())
-			.setPartPrefix(PART_PREFIX)
-			.setPendingPrefix("")
-			.setPendingSuffix("");
-
-		OneInputStreamOperatorTestHarness<Tuple2<Integer, String>, Object> testHarness =
-			createTestSink(sink, 1, 0);
-
-		testHarness.setProcessingTime(0L);
-
-		testHarness.setup();
-		testHarness.open();
-
-		for (int i = 0; i < numElements; i++) {
-			testHarness.processElement(new StreamRecord<>(Tuple2.of(
-				i, "message #" + Integer.toString(i)
-			)));
-		}
-
-		testHarness.close();
-
-		GenericData.setStringType(valueSchema, GenericData.StringType.String);
-		Schema elementSchema = AvroKeyValueSinkWriter.AvroKeyValue.getSchema(keySchema, valueSchema);
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/" + PART_PREFIX + "-0-0"));
-
-		SpecificDatumReader<GenericRecord> elementReader = new SpecificDatumReader<>(elementSchema);
-		DataFileStream<GenericRecord> dataFileStream = new DataFileStream<>(inStream, elementReader);
-		for (int i = 0; i < numElements; i++) {
-			AvroKeyValueSinkWriter.AvroKeyValue<Integer, String> wrappedEntry =
-				new AvroKeyValueSinkWriter.AvroKeyValue<>(dataFileStream.next());
-			int key = wrappedEntry.getKey();
-			Assert.assertEquals(i, key);
-			String value = wrappedEntry.getValue();
-			Assert.assertEquals("message #" + i, value);
-		}
-
-		dataFileStream.close();
-		inStream.close();
-	}
-
-	/**
-	 * This uses {@link DateTimeBucketer} to
-	 * produce rolling files. We use {@link OneInputStreamOperatorTestHarness} to manually
-	 * advance processing time.
-	 */
-	@Test
-	public void testDateTimeRollingStringWriter() throws Exception {
-		final int numElements = 20;
-
-		final String outPath = hdfsURI + "/rolling-out";
-
-		BucketingSink<String> sink = new BucketingSink<String>(outPath)
-			.setBucketer(new DateTimeBucketer<String>("ss"))
-			.setPartPrefix(PART_PREFIX)
-			.setPendingPrefix("")
-			.setPendingSuffix("");
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(sink, 1, 0);
-
-		testHarness.setProcessingTime(0L);
-
-		testHarness.setup();
-		testHarness.open();
-
-		for (int i = 0; i < numElements; i++) {
-			// Every 5 elements, increase the clock time. We should end up with 5 elements per bucket.
-			if (i % 5 == 0) {
-				testHarness.setProcessingTime(i * 1000L);
-			}
-			testHarness.processElement(new StreamRecord<>("message #" + Integer.toString(i)));
-		}
-
-		testHarness.close();
-
-		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(outPath), true);
-
-		// We should have 4 rolling files across 4 time intervals
-		int numFiles = 0;
-		while (files.hasNext()) {
-			LocatedFileStatus file = files.next();
-			numFiles++;
-			if (file.getPath().toString().contains("rolling-out/00")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 0; i < 5; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else if (file.getPath().toString().contains("rolling-out/05")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 5; i < 10; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else if (file.getPath().toString().contains("rolling-out/10")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 10; i < 15; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else if (file.getPath().toString().contains("rolling-out/15")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 15; i < 20; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else {
-				Assert.fail("File " + file + " does not match any expected roll pattern.");
-			}
-		}
-
-		Assert.assertEquals(4, numFiles);
-	}
-
-	/**
-	 * This uses a custom bucketing function which determines the bucket from the input.
-	 */
-	@Test
-	public void testCustomBucketing() throws Exception {
-		File dataDir = tempFolder.newFolder();
-
-		final int numIds = 4;
-		final int numElements = 20;
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(dataDir, 1, 0);
-
-		testHarness.setProcessingTime(0L);
-
-		testHarness.setup();
-		testHarness.open();
-
-		for (int i = 0; i < numElements; i++) {
-			testHarness.processElement(new StreamRecord<>(Integer.toString(i % numIds)));
-		}
-
-		testHarness.close();
-
-		// we should have 4 buckets, with 1 file each
-		int numFiles = 0;
-		for (File file: FileUtils.listFiles(dataDir, null, true)) {
-			if (file.getName().startsWith(PART_PREFIX)) {
-				numFiles++;
-			}
-		}
-
-		Assert.assertEquals(4, numFiles);
-	}
-
-	/**
-	 * This uses a custom bucketing function which determines the bucket from the input.
-	 * We use a simulated clock to reduce the number of buckets being written to over time.
-	 * This causes buckets to become 'inactive' and their file parts 'closed' by the sink.
-	 */
-	@Test
-	public void testCustomBucketingInactiveBucketCleanup() throws Exception {
-		File dataDir = tempFolder.newFolder();
-
-		final int step1NumIds = 4;
-		final int step2NumIds = 2;
-		final int numElementsPerStep = 20;
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(dataDir, 1, 0);
-
-		testHarness.setProcessingTime(0L);
-
-		testHarness.setup();
-		testHarness.open();
-
-		for (int i = 0; i < numElementsPerStep; i++) {
-			testHarness.processElement(new StreamRecord<>(Integer.toString(i % step1NumIds)));
-		}
-
-		testHarness.setProcessingTime(2*60*1000L);
-
-		for (int i = 0; i < numElementsPerStep; i++) {
-			testHarness.processElement(new StreamRecord<>(Integer.toString(i % step2NumIds)));
-		}
-
-		testHarness.setProcessingTime(6*60*1000L);
-
-		for (int i = 0; i < numElementsPerStep; i++) {
-			testHarness.processElement(new StreamRecord<>(Integer.toString(i % step2NumIds)));
-		}
-
-		// we should have 4 buckets, with 1 file each
-		// 2 of these buckets should have been finalised due to becoming inactive
-		int numFiles = 0;
-		int numInProgress = 0;
-		for (File file: FileUtils.listFiles(dataDir, null, true)) {
-			if (file.getAbsolutePath().endsWith("crc")) {
-				continue;
-			}
-			if (file.getPath().endsWith(IN_PROGRESS_SUFFIX)) {
-				numInProgress++;
-			}
-			numFiles++;
-		}
-
-		testHarness.close();
-
-		Assert.assertEquals(4, numFiles);
-		Assert.assertEquals(2, numInProgress);
-	}
-
-	/**
-	 * This tests user defined hdfs configuration
-	 * @throws Exception
-	 */
-	@Test
-	public void testUserDefinedConfiguration() throws Exception {
-		final String outPath = hdfsURI + "/string-non-rolling-with-config";
-		final int numElements = 20;
-
-		Map<String, String> properties = new HashMap<>();
-		Schema keySchema = Schema.create(Schema.Type.INT);
-		Schema valueSchema = Schema.create(Schema.Type.STRING);
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema.toString());
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema.toString());
-		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS, String.valueOf(true));
-		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS_CODEC, DataFileConstants.SNAPPY_CODEC);
-
-		Configuration conf = new Configuration();
-		conf.set("io.file.buffer.size", "40960");
-
-		BucketingSink<Tuple2<Integer,String>> sink = new BucketingSink<Tuple2<Integer, String>>(outPath)
-			.setFSConfig(conf)
-			.setWriter(new StreamWriterWithConfigCheck<Integer, String>(properties, "io.file.buffer.size", "40960"))
-			.setBucketer(new BasePathBucketer<Tuple2<Integer,String>>())
-			.setPartPrefix(PART_PREFIX)
-			.setPendingPrefix("")
-			.setPendingSuffix("");
-
-		OneInputStreamOperatorTestHarness<Tuple2<Integer, String>, Object> testHarness =
-			createTestSink(sink, 1, 0);
-
-		testHarness.setProcessingTime(0L);
-
-		testHarness.setup();
-		testHarness.open();
-
-		for (int i = 0; i < numElements; i++) {
-			testHarness.processElement(new StreamRecord<>(Tuple2.of(
-				i, "message #" + Integer.toString(i)
-			)));
-		}
-
-		testHarness.close();
-
-		GenericData.setStringType(valueSchema, GenericData.StringType.String);
-		Schema elementSchema = AvroKeyValueSinkWriter.AvroKeyValue.getSchema(keySchema, valueSchema);
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/" + PART_PREFIX + "-0-0"));
-
-		SpecificDatumReader<GenericRecord> elementReader = new SpecificDatumReader<>(elementSchema);
-		DataFileStream<GenericRecord> dataFileStream = new DataFileStream<>(inStream, elementReader);
-		for (int i = 0; i < numElements; i++) {
-			AvroKeyValueSinkWriter.AvroKeyValue<Integer, String> wrappedEntry =
-				new AvroKeyValueSinkWriter.AvroKeyValue<>(dataFileStream.next());
-			int key = wrappedEntry.getKey();
-			Assert.assertEquals(i, key);
-			String value = wrappedEntry.getValue();
-			Assert.assertEquals("message #" + i, value);
-		}
-
-		dataFileStream.close();
-		inStream.close();
-	}
-
-	private static class StreamWriterWithConfigCheck<K, V> extends AvroKeyValueSinkWriter<K, V> {
-		private Map<String, String> properties;
-		private String key;
-		private String expect;
-		public StreamWriterWithConfigCheck(Map<String, String> properties, String key, String expect) {
-			super(properties);
-			this.properties = properties;
-			this.key = key;
-			this.expect = expect;
-		}
-
-		@Override
-		public void open(FileSystem fs, Path path) throws IOException {
-			super.open(fs, path);
-			Assert.assertEquals(expect, fs.getConf().get(key));
-		}
-
-		@Override
-		public Writer<Tuple2<K, V>> duplicate() {
-			return new StreamWriterWithConfigCheck<>(properties, key, expect);
-		}
-	}
-
-}

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

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


[35/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
new file mode 100644
index 0000000..6235449
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import kafka.admin.AdminUtils;
+import kafka.api.PartitionMetadata;
+import kafka.common.KafkaException;
+import kafka.network.SocketServer;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.curator.test.TestingServer;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader;
+import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
+import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.util.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.Seq;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.BindException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * An implementation of the KafkaServerProvider for Kafka 0.8
+ */
+public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class);
+	private File tmpZkDir;
+	private File tmpKafkaParent;
+	private List<File> tmpKafkaDirs;
+	private List<KafkaServer> brokers;
+	private TestingServer zookeeper;
+	private String zookeeperConnectionString;
+	private String brokerConnectionString = "";
+	private Properties standardProps;
+	private Properties additionalServerProperties;
+
+	public String getBrokerConnectionString() {
+		return brokerConnectionString;
+	}
+
+	@Override
+	public Properties getStandardProperties() {
+		return standardProps;
+	}
+
+	@Override
+	public Properties getSecureProperties() {
+		return null;
+	}
+
+	@Override
+	public String getVersion() {
+		return "0.8";
+	}
+
+	@Override
+	public List<KafkaServer> getBrokers() {
+		return brokers;
+	}
+
+	@Override
+	public <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, KeyedDeserializationSchema<T> readSchema, Properties props) {
+		return new FlinkKafkaConsumer08<>(topics, readSchema, props);
+	}
+
+	@Override
+	public <T> StreamSink<T> getProducerSink(
+			String topic,
+			KeyedSerializationSchema<T> serSchema,
+			Properties props,
+			KafkaPartitioner<T> partitioner) {
+		FlinkKafkaProducer08<T> prod = new FlinkKafkaProducer08<>(
+				topic,
+				serSchema,
+				props,
+				partitioner);
+		prod.setFlushOnCheckpoint(true);
+		return new StreamSink<>(prod);
+	}
+
+	@Override
+	public <T> DataStreamSink<T> produceIntoKafka(DataStream<T> stream, String topic, KeyedSerializationSchema<T> serSchema, Properties props, KafkaPartitioner<T> partitioner) {
+		FlinkKafkaProducer08<T> prod = new FlinkKafkaProducer08<>(topic, serSchema, props, partitioner);
+		prod.setFlushOnCheckpoint(true);
+		return stream.addSink(prod);
+	}
+
+	@Override
+	public KafkaOffsetHandler createOffsetHandler(Properties props) {
+		return new KafkaOffsetHandlerImpl(props);
+	}
+
+	@Override
+	public void restartBroker(int leaderId) throws Exception {
+		brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId)));
+	}
+
+	@Override
+	public int getLeaderToShutDown(String topic) throws Exception {
+		ZkClient zkClient = createZkClient();
+		PartitionMetadata firstPart = null;
+		do {
+			if (firstPart != null) {
+				LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
+				// not the first try. Sleep a bit
+				Thread.sleep(150);
+			}
+
+			Seq<PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata();
+			firstPart = partitionMetadata.head();
+		}
+		while (firstPart.errorCode() != 0);
+		zkClient.close();
+
+		return firstPart.leader().get().id();
+	}
+
+	@Override
+	public int getBrokerId(KafkaServer server) {
+		return server.socketServer().brokerId();
+	}
+
+	@Override
+	public boolean isSecureRunSupported() {
+		return false;
+	}
+
+
+	@Override
+	public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) {
+		this.additionalServerProperties = additionalServerProperties;
+		File tempDir = new File(System.getProperty("java.io.tmpdir"));
+
+		tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
+		try {
+			Files.createDirectories(tmpZkDir.toPath());
+		} catch (IOException e) {
+			fail("cannot create zookeeper temp dir: " + e.getMessage());
+		}
+
+		tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir" + (UUID.randomUUID().toString()));
+		try {
+			Files.createDirectories(tmpKafkaParent.toPath());
+		} catch (IOException e) {
+			fail("cannot create kafka temp dir: " + e.getMessage());
+		}
+
+		tmpKafkaDirs = new ArrayList<>(numKafkaServers);
+		for (int i = 0; i < numKafkaServers; i++) {
+			File tmpDir = new File(tmpKafkaParent, "server-" + i);
+			assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
+			tmpKafkaDirs.add(tmpDir);
+		}
+
+		zookeeper = null;
+		brokers = null;
+
+		try {
+			LOG.info("Starting Zookeeper");
+			zookeeper = new TestingServer(-1, tmpZkDir);
+			zookeeperConnectionString = zookeeper.getConnectString();
+
+			LOG.info("Starting KafkaServer");
+			brokers = new ArrayList<>(numKafkaServers);
+
+			for (int i = 0; i < numKafkaServers; i++) {
+				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i)));
+				SocketServer socketServer = brokers.get(i).socketServer();
+
+				String host = socketServer.host() == null ? "localhost" : socketServer.host();
+				brokerConnectionString += hostAndPortToUrlString(host, socketServer.port()) + ",";
+			}
+
+			LOG.info("ZK and KafkaServer started.");
+		}
+		catch (Throwable t) {
+			t.printStackTrace();
+			fail("Test setup failed: " + t.getMessage());
+		}
+
+		standardProps = new Properties();
+		standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
+		standardProps.setProperty("bootstrap.servers", brokerConnectionString);
+		standardProps.setProperty("group.id", "flink-tests");
+		standardProps.setProperty("auto.commit.enable", "false");
+		standardProps.setProperty("zookeeper.session.timeout.ms", "30000"); // 6 seconds is default. Seems to be too small for travis.
+		standardProps.setProperty("zookeeper.connection.timeout.ms", "30000");
+		standardProps.setProperty("auto.offset.reset", "smallest"); // read from the beginning. (smallest is kafka 0.8)
+		standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
+	}
+
+	@Override
+	public void shutdown() {
+		if (brokers != null) {
+			for (KafkaServer broker : brokers) {
+				if (broker != null) {
+					broker.shutdown();
+				}
+			}
+			brokers.clear();
+		}
+
+		if (zookeeper != null) {
+			try {
+				zookeeper.stop();
+				zookeeper.close();
+			}
+			catch (Exception e) {
+				LOG.warn("ZK.stop() failed", e);
+			}
+			zookeeper = null;
+		}
+
+		// clean up the temp spaces
+
+		if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpKafkaParent);
+			}
+			catch (Exception e) {
+				// ignore
+			}
+		}
+		if (tmpZkDir != null && tmpZkDir.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpZkDir);
+			}
+			catch (Exception e) {
+				// ignore
+			}
+		}
+	}
+
+	@Override
+	public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor, Properties topicConfig) {
+		// create topic with one client
+		LOG.info("Creating topic {}", topic);
+
+		ZkClient creator = createZkClient();
+
+		AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig);
+		creator.close();
+
+		// validate that the topic has been created
+		final long deadline = System.currentTimeMillis() + 30000;
+		do {
+			try {
+				Thread.sleep(100);
+			}
+			catch (InterruptedException e) {
+				// restore interrupted state
+			}
+			List<KafkaTopicPartitionLeader> partitions = FlinkKafkaConsumer08.getPartitionsForTopic(Collections.singletonList(topic), standardProps);
+			if (partitions != null && partitions.size() > 0) {
+				return;
+			}
+		}
+		while (System.currentTimeMillis() < deadline);
+		fail ("Test topic could not be created");
+	}
+
+	@Override
+	public void deleteTestTopic(String topic) {
+		LOG.info("Deleting topic {}", topic);
+
+		ZkClient zk = createZkClient();
+		AdminUtils.deleteTopic(zk, topic);
+		zk.close();
+	}
+
+	private ZkClient createZkClient() {
+		return new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
+				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
+	}
+
+	/**
+	 * Only for the 0.8 server we need access to the zk client.
+	 */
+	public CuratorFramework createCuratorClient() {
+		RetryPolicy retryPolicy = new ExponentialBackoffRetry(100, 10);
+		CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(standardProps.getProperty("zookeeper.connect"), retryPolicy);
+		curatorClient.start();
+		return curatorClient;
+	}
+
+	/**
+	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
+	 */
+	protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception {
+		LOG.info("Starting broker with id {}", brokerId);
+		Properties kafkaProperties = new Properties();
+
+		// properties have to be Strings
+		kafkaProperties.put("advertised.host.name", KAFKA_HOST);
+		kafkaProperties.put("broker.id", Integer.toString(brokerId));
+		kafkaProperties.put("log.dir", tmpFolder.toString());
+		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
+		kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
+		kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
+
+		// for CI stability, increase zookeeper session timeout
+		kafkaProperties.put("zookeeper.session.timeout.ms", "30000");
+		kafkaProperties.put("zookeeper.connection.timeout.ms", "30000");
+		if(additionalServerProperties != null) {
+			kafkaProperties.putAll(additionalServerProperties);
+		}
+
+		final int numTries = 5;
+
+		for (int i = 1; i <= numTries; i++) {
+			int kafkaPort = NetUtils.getAvailablePort();
+			kafkaProperties.put("port", Integer.toString(kafkaPort));
+			KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
+
+			try {
+				KafkaServer server = new KafkaServer(kafkaConfig, new KafkaLocalSystemTime());
+				server.startup();
+				return server;
+			}
+			catch (KafkaException e) {
+				if (e.getCause() instanceof BindException) {
+					// port conflict, retry...
+					LOG.info("Port conflict when starting Kafka Broker. Retrying...");
+				}
+				else {
+					throw e;
+				}
+			}
+		}
+
+		throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
+	}
+
+	private class KafkaOffsetHandlerImpl implements KafkaOffsetHandler {
+
+		private final CuratorFramework offsetClient;
+		private final String groupId;
+
+		public KafkaOffsetHandlerImpl(Properties props) {
+			offsetClient = createCuratorClient();
+			groupId = props.getProperty("group.id");
+		}
+
+		@Override
+		public Long getCommittedOffset(String topicName, int partition) {
+			try {
+				return ZookeeperOffsetHandler.getOffsetFromZooKeeper(offsetClient, groupId, topicName, partition);
+			} catch (Exception e) {
+				throw new RuntimeException("Exception when getting offsets from Zookeeper", e);
+			}
+		}
+
+		@Override
+		public void close() {
+			offsetClient.close();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java
new file mode 100644
index 0000000..6298c92
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java
@@ -0,0 +1,603 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.*;
+import static java.util.Arrays.asList;
+import static java.util.Collections.emptyList;
+import static java.util.Collections.singletonList;
+
+public class ClosableBlockingQueueTest {
+
+	// ------------------------------------------------------------------------
+	//  single-threaded unit tests
+	// ------------------------------------------------------------------------
+	
+	@Test
+	public void testCreateQueueHashCodeEquals() {
+		try {
+			ClosableBlockingQueue<String> queue1 = new ClosableBlockingQueue<>();
+			ClosableBlockingQueue<String> queue2 = new ClosableBlockingQueue<>(22);
+
+			assertTrue(queue1.isOpen());
+			assertTrue(queue2.isOpen());
+			assertTrue(queue1.isEmpty());
+			assertTrue(queue2.isEmpty());
+			assertEquals(0, queue1.size());
+			assertEquals(0, queue2.size());
+			
+			assertTrue(queue1.hashCode() == queue2.hashCode());
+			//noinspection EqualsWithItself
+			assertTrue(queue1.equals(queue1));
+			//noinspection EqualsWithItself
+			assertTrue(queue2.equals(queue2));
+			assertTrue(queue1.equals(queue2));
+			
+			assertNotNull(queue1.toString());
+			assertNotNull(queue2.toString());
+
+			List<String> elements = new ArrayList<>();
+			elements.add("a");
+			elements.add("b");
+			elements.add("c");
+
+			ClosableBlockingQueue<String> queue3 = new ClosableBlockingQueue<>(elements);
+			ClosableBlockingQueue<String> queue4 = new ClosableBlockingQueue<>(asList("a", "b", "c"));
+
+			assertTrue(queue3.isOpen());
+			assertTrue(queue4.isOpen());
+			assertFalse(queue3.isEmpty());
+			assertFalse(queue4.isEmpty());
+			assertEquals(3, queue3.size());
+			assertEquals(3, queue4.size());
+
+			assertTrue(queue3.hashCode() == queue4.hashCode());
+			//noinspection EqualsWithItself
+			assertTrue(queue3.equals(queue3));
+			//noinspection EqualsWithItself
+			assertTrue(queue4.equals(queue4));
+			assertTrue(queue3.equals(queue4));
+			
+			assertNotNull(queue3.toString());
+			assertNotNull(queue4.toString());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCloseEmptyQueue() {
+		try {
+			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
+			assertTrue(queue.isOpen());
+			assertTrue(queue.close());
+			assertFalse(queue.isOpen());
+			
+			assertFalse(queue.addIfOpen("element"));
+			assertTrue(queue.isEmpty());
+			
+			try {
+				queue.add("some element");
+				fail("should cause an exception");
+			} catch (IllegalStateException ignored) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testCloseNonEmptyQueue() {
+		try {
+			ClosableBlockingQueue<Integer> queue = new ClosableBlockingQueue<>(asList(1, 2, 3));
+			assertTrue(queue.isOpen());
+			
+			assertFalse(queue.close());
+			assertFalse(queue.close());
+			
+			queue.poll();
+
+			assertFalse(queue.close());
+			assertFalse(queue.close());
+			
+			queue.pollBatch();
+
+			assertTrue(queue.close());
+			assertFalse(queue.isOpen());
+
+			assertFalse(queue.addIfOpen(42));
+			assertTrue(queue.isEmpty());
+
+			try {
+				queue.add(99);
+				fail("should cause an exception");
+			} catch (IllegalStateException ignored) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testPeekAndPoll() {
+		try {
+			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
+			
+			assertNull(queue.peek());
+			assertNull(queue.peek());
+			assertNull(queue.poll());
+			assertNull(queue.poll());
+			
+			assertEquals(0, queue.size());
+			
+			queue.add("a");
+			queue.add("b");
+			queue.add("c");
+
+			assertEquals(3, queue.size());
+			
+			assertEquals("a", queue.peek());
+			assertEquals("a", queue.peek());
+			assertEquals("a", queue.peek());
+
+			assertEquals(3, queue.size());
+			
+			assertEquals("a", queue.poll());
+			assertEquals("b", queue.poll());
+
+			assertEquals(1, queue.size());
+			
+			assertEquals("c", queue.peek());
+			assertEquals("c", queue.peek());
+
+			assertEquals("c", queue.poll());
+
+			assertEquals(0, queue.size());
+			assertNull(queue.poll());
+			assertNull(queue.peek());
+			assertNull(queue.peek());
+			
+			assertTrue(queue.close());
+			
+			try {
+				queue.peek();
+				fail("should cause an exception");
+			} catch (IllegalStateException ignored) {
+				// expected
+			}
+
+			try {
+				queue.poll();
+				fail("should cause an exception");
+			} catch (IllegalStateException ignored) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testPollBatch() {
+		try {
+			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
+
+			assertNull(queue.pollBatch());
+			
+			queue.add("a");
+			queue.add("b");
+			
+			assertEquals(asList("a", "b"), queue.pollBatch());
+			assertNull(queue.pollBatch());
+			
+			queue.add("c");
+
+			assertEquals(singletonList("c"), queue.pollBatch());
+			assertNull(queue.pollBatch());
+
+			assertTrue(queue.close());
+
+			try {
+				queue.pollBatch();
+				fail("should cause an exception");
+			} catch (IllegalStateException ignored) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testGetElementBlocking() {
+		try {
+			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
+
+			assertNull(queue.getElementBlocking(1));
+			assertNull(queue.getElementBlocking(3));
+			assertNull(queue.getElementBlocking(2));
+
+			assertEquals(0, queue.size());
+
+			queue.add("a");
+			queue.add("b");
+			queue.add("c");
+			queue.add("d");
+			queue.add("e");
+			queue.add("f");
+
+			assertEquals(6, queue.size());
+
+			assertEquals("a", queue.getElementBlocking(99));
+			assertEquals("b", queue.getElementBlocking());
+
+			assertEquals(4, queue.size());
+
+			assertEquals("c", queue.getElementBlocking(0));
+			assertEquals("d", queue.getElementBlocking(1000000));
+			assertEquals("e", queue.getElementBlocking());
+			assertEquals("f", queue.getElementBlocking(1786598));
+
+			assertEquals(0, queue.size());
+
+			assertNull(queue.getElementBlocking(1));
+			assertNull(queue.getElementBlocking(3));
+			assertNull(queue.getElementBlocking(2));
+
+			assertTrue(queue.close());
+
+			try {
+				queue.getElementBlocking();
+				fail("should cause an exception");
+			} catch (IllegalStateException ignored) {
+				// expected
+			}
+
+			try {
+				queue.getElementBlocking(1000000000L);
+				fail("should cause an exception");
+			} catch (IllegalStateException ignored) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testGetBatchBlocking() {
+		try {
+			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
+
+			assertEquals(emptyList(), queue.getBatchBlocking(1));
+			assertEquals(emptyList(), queue.getBatchBlocking(3));
+			assertEquals(emptyList(), queue.getBatchBlocking(2));
+
+			queue.add("a");
+			queue.add("b");
+
+			assertEquals(asList("a", "b"), queue.getBatchBlocking(900000009));
+
+			queue.add("c");
+			queue.add("d");
+
+			assertEquals(asList("c", "d"), queue.getBatchBlocking());
+
+			assertEquals(emptyList(), queue.getBatchBlocking(2));
+
+			queue.add("e");
+
+			assertEquals(singletonList("e"), queue.getBatchBlocking(0));
+
+			queue.add("f");
+
+			assertEquals(singletonList("f"), queue.getBatchBlocking(1000000000));
+
+			assertEquals(0, queue.size());
+
+			assertEquals(emptyList(), queue.getBatchBlocking(1));
+			assertEquals(emptyList(), queue.getBatchBlocking(3));
+			assertEquals(emptyList(), queue.getBatchBlocking(2));
+
+			assertTrue(queue.close());
+
+			try {
+				queue.getBatchBlocking();
+				fail("should cause an exception");
+			} catch (IllegalStateException ignored) {
+				// expected
+			}
+
+			try {
+				queue.getBatchBlocking(1000000000L);
+				fail("should cause an exception");
+			} catch (IllegalStateException ignored) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  multi-threaded tests
+	// ------------------------------------------------------------------------
+	
+	@Test
+	public void notifyOnClose() {
+		try {
+			final long oneYear = 365L * 24 * 60 * 60 * 1000;
+			
+			// test "getBatchBlocking()"
+			final ClosableBlockingQueue<String> queue1 = new ClosableBlockingQueue<>();
+			QueueCall call1 = new QueueCall() {
+				@Override
+				public void call() throws Exception {
+					queue1.getBatchBlocking();
+				}
+			};
+			testCallExitsOnClose(call1, queue1);
+
+			// test "getBatchBlocking()"
+			final ClosableBlockingQueue<String> queue2 = new ClosableBlockingQueue<>();
+			QueueCall call2 = new QueueCall() {
+				@Override
+				public void call() throws Exception {
+					queue2.getBatchBlocking(oneYear);
+				}
+			};
+			testCallExitsOnClose(call2, queue2);
+
+			// test "getBatchBlocking()"
+			final ClosableBlockingQueue<String> queue3 = new ClosableBlockingQueue<>();
+			QueueCall call3 = new QueueCall() {
+				@Override
+				public void call() throws Exception {
+					queue3.getElementBlocking();
+				}
+			};
+			testCallExitsOnClose(call3, queue3);
+
+			// test "getBatchBlocking()"
+			final ClosableBlockingQueue<String> queue4 = new ClosableBlockingQueue<>();
+			QueueCall call4 = new QueueCall() {
+				@Override
+				public void call() throws Exception {
+					queue4.getElementBlocking(oneYear);
+				}
+			};
+			testCallExitsOnClose(call4, queue4);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+	@Test
+	public void testMultiThreadedAddGet() {
+		try {
+			final ClosableBlockingQueue<Integer> queue = new ClosableBlockingQueue<>();
+			final AtomicReference<Throwable> pushErrorRef = new AtomicReference<>();
+			final AtomicReference<Throwable> pollErrorRef = new AtomicReference<>();
+			
+			final int numElements = 2000;
+			
+			Thread pusher = new Thread("pusher") {
+
+				@Override
+				public void run() {
+					try {
+						final Random rnd = new Random();
+						for (int i = 0; i < numElements; i++) {
+							queue.add(i);
+							
+							// sleep a bit, sometimes
+							int sleepTime = rnd.nextInt(3);
+							if (sleepTime > 1) {
+								Thread.sleep(sleepTime);
+							}
+						}
+						
+						while (true) {
+							if (queue.close()) {
+								break;
+							} else {
+								Thread.sleep(5);
+							}
+						}
+					} catch (Throwable t) {
+						pushErrorRef.set(t);
+					}
+				}
+			};
+			pusher.start();
+
+			Thread poller = new Thread("poller") {
+
+				@SuppressWarnings("InfiniteLoopStatement")
+				@Override
+				public void run() {
+					try {
+						int count = 0;
+						
+						try {
+							final Random rnd = new Random();
+							int nextExpected = 0;
+							
+							while (true) {
+								int getMethod = count % 7;
+								switch (getMethod) {
+									case 0: {
+										Integer next = queue.getElementBlocking(1);
+										if (next != null) {
+											assertEquals(nextExpected, next.intValue());
+											nextExpected++;
+											count++;
+										}
+										break;
+									}
+									case 1: {
+										List<Integer> nextList = queue.getBatchBlocking();
+										for (Integer next : nextList) {
+											assertNotNull(next);
+											assertEquals(nextExpected, next.intValue());
+											nextExpected++;
+											count++;
+										}
+										break;
+									}
+									case 2: {
+										List<Integer> nextList = queue.getBatchBlocking(1);
+										if (nextList != null) {
+											for (Integer next : nextList) {
+												assertNotNull(next);
+												assertEquals(nextExpected, next.intValue());
+												nextExpected++;
+												count++;
+											}
+										}
+										break;
+									}
+									case 3: {
+										Integer next = queue.poll();
+										if (next != null) {
+											assertEquals(nextExpected, next.intValue());
+											nextExpected++;
+											count++;
+										}
+										break;
+									}
+									case 4: {
+										List<Integer> nextList = queue.pollBatch();
+										if (nextList != null) {
+											for (Integer next : nextList) {
+												assertNotNull(next);
+												assertEquals(nextExpected, next.intValue());
+												nextExpected++;
+												count++;
+											}
+										}
+										break;
+									}
+									default: {
+										Integer next = queue.getElementBlocking();
+										assertNotNull(next);
+										assertEquals(nextExpected, next.intValue());
+										nextExpected++;
+										count++;
+									}
+								}
+								
+								// sleep a bit, sometimes
+								int sleepTime = rnd.nextInt(3);
+								if (sleepTime > 1) {
+									Thread.sleep(sleepTime);
+								}
+							}
+						} catch (IllegalStateException e) {
+							// we get this once the queue is closed
+							assertEquals(numElements, count);
+						}
+					} catch (Throwable t) {
+						pollErrorRef.set(t);
+					}
+				}
+			};
+			poller.start();
+			
+			pusher.join();
+			poller.join();
+			
+			if (pushErrorRef.get() != null) {
+				Throwable t = pushErrorRef.get();
+				t.printStackTrace();
+				fail("Error in pusher: " + t.getMessage());
+			}
+			if (pollErrorRef.get() != null) {
+				Throwable t = pollErrorRef.get();
+				t.printStackTrace();
+				fail("Error in poller: " + t.getMessage());
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Utils
+	// ------------------------------------------------------------------------
+	
+	private static void testCallExitsOnClose(
+			final QueueCall call, ClosableBlockingQueue<String> queue) throws Exception {
+		
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+		
+		Runnable runnable = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					call.call();
+				} catch (Throwable t) {
+					errorRef.set(t);
+				}
+			}
+		};
+
+		Thread thread = new Thread(runnable);
+		thread.start();
+		Thread.sleep(100);
+		queue.close();
+		thread.join();
+
+		@SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+		Throwable cause = errorRef.get();
+		assertTrue(cause instanceof IllegalStateException);
+	}
+	
+	private interface QueueCall {
+		void call() throws Exception;
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml
new file mode 100644
index 0000000..3894499
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml
@@ -0,0 +1,212 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-kafka-0.9_2.10</artifactId>
+	<name>flink-connector-kafka-0.9</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<kafka.version>0.9.0.1</kafka.version>
+	</properties>
+
+	<dependencies>
+
+		<!-- core dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<version>${project.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>org.apache.kafka</groupId>
+					<artifactId>kafka_${scala.binary.version}</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project,
+			won't depend on flink-table. -->
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.kafka</groupId>
+			<artifactId>kafka-clients</artifactId>
+			<version>${kafka.version}</version>
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<version>${project.version}</version>
+			<exclusions>
+				<!-- exclude 0.8 dependencies -->
+				<exclusion>
+					<groupId>org.apache.kafka</groupId>
+					<artifactId>kafka_${scala.binary.version}</artifactId>
+				</exclusion>
+			</exclusions>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- include 0.9 server for tests  -->
+			<groupId>org.apache.kafka</groupId>
+			<artifactId>kafka_${scala.binary.version}</artifactId>
+			<version>${kafka.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-metrics-jmx</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+		
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-minikdc</artifactId>
+			<version>${minikdc.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+						<configuration>
+							<includes>
+								<include>**/KafkaTestEnvironmentImpl*</include>
+							</includes>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-source-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>attach-test-sources</id>
+						<goals>
+							<goal>test-jar-no-fork</goal>
+						</goals>
+						<configuration>
+							<includes>
+								<include>**/KafkaTestEnvironmentImpl*</include>
+							</includes>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
+					<forkCount>1</forkCount>
+					<argLine>-Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
+				</configuration>
+			</plugin>
+			<!--
+            https://issues.apache.org/jira/browse/DIRSHARED-134
+            Required to pull the Mini-KDC transitive dependency
+            -->
+			<plugin>
+				<groupId>org.apache.felix</groupId>
+				<artifactId>maven-bundle-plugin</artifactId>
+				<version>3.0.1</version>
+				<inherited>true</inherited>
+				<extensions>true</extensions>
+			</plugin>
+		</plugins>
+	</build>
+	
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
new file mode 100644
index 0000000..29bb8e4
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
@@ -0,0 +1,269 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.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.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.KafkaConsumer;
+import org.apache.kafka.common.PartitionInfo;
+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.List;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
+ * Apache Kafka 0.9.x. The consumer can run in multiple parallel instances, each of which will pull
+ * data from one or more Kafka partitions. 
+ * 
+ * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
+ * during a failure, and that the computation processes elements "exactly once". 
+ * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
+ *
+ * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
+ * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
+ * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
+ * has consumed a topic.</p>
+ *
+ * <p>Please refer to Kafka's documentation for the available configuration properties:
+ * http://kafka.apache.org/documentation.html#newconsumerconfigs</p>
+ *
+ * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
+ * is constructed. That means that the client that submits the program needs to be able to
+ * reach the Kafka brokers or ZooKeeper.</p>
+ */
+public class 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 **/
+	public static final String KEY_POLL_TIMEOUT = "flink.poll-timeout";
+
+
+	/** 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 **/
+	protected final Properties properties;
+
+	/** 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 */
+	protected final long pollTimeout;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.9.x
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
+	public FlinkKafkaConsumer09(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
+		this(Collections.singletonList(topic), valueDeserializer, props);
+	}
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.9.x
+	 *
+	 * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value
+	 * pairs, offsets, and topic names from Kafka.
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param deserializer
+	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
+	public FlinkKafkaConsumer09(String topic, KeyedDeserializationSchema<T> deserializer, Properties props) {
+		this(Collections.singletonList(topic), deserializer, props);
+	}
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.9.x
+	 *
+	 * This constructor allows passing multiple topics to the consumer.
+	 *
+	 * @param topics
+	 *           The Kafka topics to read from.
+	 * @param deserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties that are used to configure both the fetcher and the offset handler.
+	 */
+	public FlinkKafkaConsumer09(List<String> topics, DeserializationSchema<T> deserializer, Properties props) {
+		this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props);
+	}
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.9.x
+	 *
+	 * This constructor allows passing multiple topics and a key/value deserialization schema.
+	 *
+	 * @param topics
+	 *           The Kafka topics to read from.
+	 * @param deserializer
+	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           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(topics, deserializer);
+
+		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);
+		}
+	}
+
+	@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.getProcessingTimeService(),
+				runtimeContext.getExecutionConfig().getAutoWatermarkInterval(),
+				runtimeContext.getUserCodeClassLoader(),
+				runtimeContext.isCheckpointingEnabled(),
+				runtimeContext.getTaskNameWithSubtasks(),
+				runtimeContext.getMetricGroup(),
+				deserializer,
+				properties,
+				pollTimeout,
+				useMetrics);
+		
+	}
+
+	@Override
+	protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
+		// read the partitions that belong to the listed topics
+		final List<KafkaTopicPartition> partitions = new ArrayList<>();
+
+		try (KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(this.properties)) {
+			for (final String topic: topics) {
+				// get partitions for each topic
+				List<PartitionInfo> partitionsForTopic = consumer.partitionsFor(topic);
+				// for non existing topics, the list might be null.
+				if (partitionsForTopic != null) {
+					partitions.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic));
+				}
+			}
+		}
+
+		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: {}", partitions.size(), topics);
+
+		if (LOG.isInfoEnabled()) {
+			logPartitionInfo(LOG, partitions);
+		}
+
+		return partitions;
+	}
+
+	// ------------------------------------------------------------------------
+	//  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
+	 */
+	private static List<KafkaTopicPartition> convertToFlinkKafkaTopicPartition(List<PartitionInfo> partitions) {
+		checkNotNull(partitions);
+
+		List<KafkaTopicPartition> ret = new ArrayList<>(partitions.size());
+		for (PartitionInfo pi : partitions) {
+			ret.add(new KafkaTopicPartition(pi.topic(), pi.partition()));
+		}
+		return ret;
+	}
+
+	/**
+	 * Makes sure that the ByteArrayDeserializer is registered in the Kafka properties.
+	 * 
+	 * @param props The Kafka properties to register the serializer in.
+	 */
+	private static void setDeserializer(Properties props) {
+		final String deSerName = ByteArrayDeserializer.class.getCanonicalName();
+
+		Object keyDeSer = props.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
+		Object valDeSer = props.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
+
+		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);
+		}
+
+		props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deSerName);
+		props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deSerName);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
new file mode 100644
index 0000000..2a3e39d
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import java.util.Properties;
+
+
+/**
+ * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.9.
+ *
+ * Please note that this producer does not have any reliability guarantees.
+ *
+ * @param <IN> Type of the messages to write into Kafka.
+ */
+public class FlinkKafkaProducer09<IN> extends FlinkKafkaProducerBase<IN> {
+
+	private static final long serialVersionUID = 1L;
+
+	// ------------------- Keyless serialization schema constructors ----------------------
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param brokerList
+	 *			Comma separated addresses of the brokers
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined (keyless) serialization schema.
+	 */
+	public FlinkKafkaProducer09(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
+		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner<IN>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined (keyless) serialization schema.
+	 * @param producerConfig
+	 * 			Properties with the producer configuration.
+	 */
+	public FlinkKafkaProducer09(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
+		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner<IN>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param topicId The topic to write data to
+	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
+	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
+	 */
+	public FlinkKafkaProducer09(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
+		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
+
+	}
+
+	// ------------------- Key/Value serialization schema constructors ----------------------
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param brokerList
+	 *			Comma separated addresses of the brokers
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined serialization schema supporting key/value messages
+	 */
+	public FlinkKafkaProducer09(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
+		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner<IN>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined serialization schema supporting key/value messages
+	 * @param producerConfig
+	 * 			Properties with the producer configuration.
+	 */
+	public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
+		this(topicId, serializationSchema, producerConfig, new FixedPartitioner<IN>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param topicId The topic to write data to
+	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
+	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
+	 */
+	public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
+		super(topicId, serializationSchema, producerConfig, customPartitioner);
+	}
+
+	@Override
+	protected void flush() {
+		if (this.producer != null) {
+			producer.flush();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
new file mode 100644
index 0000000..38ea47c
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Kafka 0.9 {@link KafkaTableSink} that serializes data in JSON format.
+ */
+public class Kafka09JsonTableSink extends KafkaJsonTableSink {
+	/**
+	 * Creates {@link KafkaTableSink} for Kafka 0.9
+	 *
+	 * @param topic topic in Kafka to which table is written
+	 * @param properties properties to connect to Kafka
+	 * @param partitioner Kafka partitioner
+	 */
+	public Kafka09JsonTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner) {
+		super(topic, properties, partitioner);
+	}
+
+	@Override
+	protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties, SerializationSchema<Row> serializationSchema, KafkaPartitioner<Row> partitioner) {
+		return new FlinkKafkaProducer09<>(topic, serializationSchema, properties, partitioner);
+	}
+
+	@Override
+	protected Kafka09JsonTableSink createCopy() {
+		return new Kafka09JsonTableSink(topic, properties, partitioner);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
new file mode 100644
index 0000000..975ef58
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.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;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Kafka {@link StreamTableSource} for Kafka 0.9.
+ */
+public class Kafka09JsonTableSource extends KafkaJsonTableSource {
+
+	/**
+	 * Creates a Kafka 0.9 JSON {@link StreamTableSource}.
+	 *
+	 * @param topic      Kafka topic to consume.
+	 * @param properties Properties for the Kafka consumer.
+	 * @param fieldNames Row field names.
+	 * @param fieldTypes Row field types.
+	 */
+	public Kafka09JsonTableSource(
+			String topic,
+			Properties properties,
+			String[] fieldNames,
+			TypeInformation<?>[] fieldTypes) {
+
+		super(topic, properties, fieldNames, fieldTypes);
+	}
+
+	/**
+	 * Creates a Kafka 0.9 JSON {@link StreamTableSource}.
+	 *
+	 * @param topic      Kafka topic to consume.
+	 * @param properties Properties for the Kafka consumer.
+	 * @param fieldNames Row field names.
+	 * @param fieldTypes Row field types.
+	 */
+	public Kafka09JsonTableSource(
+			String topic,
+			Properties properties,
+			String[] fieldNames,
+			Class<?>[] fieldTypes) {
+
+		super(topic, properties, fieldNames, fieldTypes);
+	}
+
+	@Override
+	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
+		return new FlinkKafkaConsumer09<>(topic, deserializationSchema, properties);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
new file mode 100644
index 0000000..03b5040
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Kafka {@link StreamTableSource} for Kafka 0.9.
+ */
+public class Kafka09TableSource extends KafkaTableSource {
+
+	/**
+	 * Creates a Kafka 0.9 {@link StreamTableSource}.
+	 *
+	 * @param topic                 Kafka topic to consume.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param deserializationSchema Deserialization schema to use for Kafka records.
+	 * @param fieldNames            Row field names.
+	 * @param fieldTypes            Row field types.
+	 */
+	public Kafka09TableSource(
+			String topic,
+			Properties properties,
+			DeserializationSchema<Row> deserializationSchema,
+			String[] fieldNames,
+			TypeInformation<?>[] fieldTypes) {
+
+		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
+	}
+
+	/**
+	 * Creates a Kafka 0.9 {@link StreamTableSource}.
+	 *
+	 * @param topic                 Kafka topic to consume.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param deserializationSchema Deserialization schema to use for Kafka records.
+	 * @param fieldNames            Row field names.
+	 * @param fieldTypes            Row field types.
+	 */
+	public Kafka09TableSource(
+			String topic,
+			Properties properties,
+			DeserializationSchema<Row> deserializationSchema,
+			String[] fieldNames,
+			Class<?>[] fieldTypes) {
+
+		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
+	}
+
+	@Override
+	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
+		return new FlinkKafkaConsumer09<>(topic, deserializationSchema, properties);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java
new file mode 100644
index 0000000..e6e3c51
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internal;
+
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.ThreadSafe;
+import java.io.Closeable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The Handover is a utility to hand over data (a buffer of records) and exception from a
+ * <i>producer</i> thread to a <i>consumer</i> thread. It effectively behaves like a
+ * "size one blocking queue", with some extras around exception reporting, closing, and
+ * waking up thread without {@link Thread#interrupt() interrupting} threads.
+ * 
+ * <p>This class is used in the Flink Kafka Consumer to hand over data and exceptions between
+ * the thread that runs the KafkaConsumer class and the main thread.
+ * 
+ * <p>The Handover has the notion of "waking up" the producer thread with a {@link WakeupException}
+ * rather than a thread interrupt.
+ * 
+ * <p>The Handover can also be "closed", signalling from one thread to the other that it
+ * the thread has terminated.
+ */
+@ThreadSafe
+public final class Handover implements Closeable {
+
+	private final Object lock = new Object();
+
+	private ConsumerRecords<byte[], byte[]> next;
+	private Throwable error;
+	private boolean wakeupProducer;
+
+	/**
+	 * Polls the next element from the Handover, possibly blocking until the next element is
+	 * available. This method behaves similar to polling from a blocking queue.
+	 * 
+	 * <p>If an exception was handed in by the producer ({@link #reportError(Throwable)}), then
+	 * that exception is thrown rather than an element being returned.
+	 * 
+	 * @return The next element (buffer of records, never null).
+	 * 
+	 * @throws ClosedException Thrown if the Handover was {@link #close() closed}.
+	 * @throws Exception Rethrows exceptions from the {@link #reportError(Throwable)} method.
+	 */
+	@Nonnull
+	public ConsumerRecords<byte[], byte[]> pollNext() throws Exception {
+		synchronized (lock) {
+			while (next == null && error == null) {
+				lock.wait();
+			}
+
+			ConsumerRecords<byte[], byte[]> n = next;
+			if (n != null) {
+				next = null;
+				lock.notifyAll();
+				return n;
+			}
+			else {
+				ExceptionUtils.rethrowException(error, error.getMessage());
+
+				// this statement cannot be reached since the above method always throws an exception
+				// this is only here to silence the compiler and any warnings
+				return ConsumerRecords.empty(); 
+			}
+		}
+	}
+
+	/**
+	 * Hands over an element from the producer. If the Handover already has an element that was
+	 * not yet picked up by the consumer thread, this call blocks until the consumer picks up that
+	 * previous element.
+	 * 
+	 * <p>This behavior is similar to a "size one" blocking queue.
+	 * 
+	 * @param element The next element to hand over.
+	 * 
+	 * @throws InterruptedException
+	 *                 Thrown, if the thread is interrupted while blocking for the Handover to be empty.
+	 * @throws WakeupException
+	 *                 Thrown, if the {@link #wakeupProducer()} method is called while blocking for
+	 *                 the Handover to be empty.
+	 * @throws ClosedException
+	 *                 Thrown if the Handover was closed or concurrently being closed.
+	 */
+	public void produce(final ConsumerRecords<byte[], byte[]> element)
+			throws InterruptedException, WakeupException, ClosedException {
+
+		checkNotNull(element);
+
+		synchronized (lock) {
+			while (next != null && !wakeupProducer) {
+				lock.wait();
+			}
+
+			wakeupProducer = false;
+
+			// if there is still an element, we must have been woken up
+			if (next != null) {
+				throw new WakeupException();
+			}
+			// if there is no error, then this is open and can accept this element
+			else if (error == null) {
+				next = element;
+				lock.notifyAll();
+			}
+			// an error marks this as closed for the producer
+			else {
+				throw new ClosedException();
+			}
+		}
+	}
+
+	/**
+	 * Reports an exception. The consumer will throw the given exception immediately, if
+	 * it is currently blocked in the {@link #pollNext()} method, or the next time it
+	 * calls that method.
+	 * 
+	 * <p>After this method has been called, no call to either {@link #produce(ConsumerRecords)}
+	 * or {@link #pollNext()} will ever return regularly any more, but will always return
+	 * exceptionally.
+	 * 
+	 * <p>If another exception was already reported, this method does nothing.
+	 * 
+	 * <p>For the producer, the Handover will appear as if it was {@link #close() closed}.
+	 * 
+	 * @param t The exception to report.
+	 */
+	public void reportError(Throwable t) {
+		checkNotNull(t);
+
+		synchronized (lock) {
+			// do not override the initial exception
+			if (error == null) {
+				error = t;
+			}
+			next = null;
+			lock.notifyAll();
+		}
+	}
+
+	/**
+	 * Closes the handover. Both the {@link #produce(ConsumerRecords)} method and the
+	 * {@link #pollNext()} will throw a {@link ClosedException} on any currently blocking and
+	 * future invocations.
+	 * 
+	 * <p>If an exception was previously reported via the {@link #reportError(Throwable)} method,
+	 * that exception will not be overridden. The consumer thread will throw that exception upon
+	 * calling {@link #pollNext()}, rather than the {@code ClosedException}.
+	 */
+	@Override
+	public void close() {
+		synchronized (lock) {
+			next = null;
+			wakeupProducer = false;
+
+			if (error == null) {
+				error = new ClosedException();
+			}
+			lock.notifyAll();
+		}
+	}
+
+	/**
+	 * Wakes the producer thread up. If the producer thread is currently blocked in
+	 * the {@link #produce(ConsumerRecords)} method, it will exit the method throwing
+	 * a {@link WakeupException}.
+	 */
+	public void wakeupProducer() {
+		synchronized (lock) {
+			wakeupProducer = true;
+			lock.notifyAll();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * An exception thrown by the Handover in the {@link #pollNext()} or
+	 * {@link #produce(ConsumerRecords)} method, after the Handover was closed via
+	 * {@link #close()}.
+	 */
+	public static final class ClosedException extends Exception {
+		private static final long serialVersionUID = 1L;
+	}
+
+	/**
+	 * A special exception thrown bv the Handover in the {@link #produce(ConsumerRecords)}
+	 * method when the producer is woken up from a blocking call via {@link #wakeupProducer()}.
+	 */
+	public static final class WakeupException extends Exception {
+		private static final long serialVersionUID = 1L;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java
new file mode 100644
index 0000000..d495327
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.metrics.MetricGroup;
+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.connectors.kafka.internals.AbstractFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+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.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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> {
+
+	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 handover of data and exceptions between the consumer thread and the task thread */
+	private final Handover handover;
+
+	/** The thread that runs the actual KafkaConsumer and hand the record batches to this fetcher */
+	private final KafkaConsumerThread consumerThread;
+
+	/** 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,
+			ProcessingTimeService processingTimeProvider,
+			long autoWatermarkInterval,
+			ClassLoader userCodeClassLoader,
+			boolean enableCheckpointing,
+			String taskNameWithSubtasks,
+			MetricGroup metricGroup,
+			KeyedDeserializationSchema<T> deserializer,
+			Properties kafkaProperties,
+			long pollTimeout,
+			boolean useMetrics) throws Exception
+	{
+		super(
+				sourceContext,
+				assignedPartitions,
+				watermarksPeriodic,
+				watermarksPunctuated,
+				processingTimeProvider,
+				autoWatermarkInterval,
+				userCodeClassLoader,
+				useMetrics);
+
+		this.deserializer = deserializer;
+		this.handover = new Handover();
+
+		final MetricGroup kafkaMetricGroup = metricGroup.addGroup("KafkaConsumer");
+		addOffsetStateGauge(kafkaMetricGroup);
+
+		// if checkpointing is enabled, we are not automatically committing to Kafka.
+		kafkaProperties.setProperty(
+				ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG,
+				Boolean.toString(!enableCheckpointing));
+		
+		this.consumerThread = new KafkaConsumerThread(
+				LOG,
+				handover,
+				kafkaProperties,
+				subscribedPartitions(),
+				kafkaMetricGroup,
+				createCallBridge(),
+				getFetcherName() + " for " + taskNameWithSubtasks,
+				pollTimeout,
+				useMetrics);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Fetcher work methods
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void runFetchLoop() throws Exception {
+		try {
+			final Handover handover = this.handover;
+
+			// kick off the actual Kafka consumer
+			consumerThread.start();
+
+			while (running) {
+				// this blocks until we get the next records
+				// it automatically re-throws exceptions encountered in the fetcher thread
+				final ConsumerRecords<byte[], byte[]> records = handover.pollNext();
+
+				// 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) {
+
+						final 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 updates offset state atomically
+						// and deals with timestamps and watermark generation
+						emitRecord(value, partition, record.offset(), record);
+					}
+				}
+			}
+		}
+		finally {
+			// this signals the consumer thread that no more work is to be done
+			consumerThread.shutdown();
+		}
+
+		// on a clean exit, wait for the runner thread
+		try {
+			consumerThread.join();
+		}
+		catch (InterruptedException e) {
+			// may be the result of a wake-up interruption after an exception.
+			// we ignore this here and only restore the interruption state
+			Thread.currentThread().interrupt();
+		}
+	}
+
+	@Override
+	public void cancel() {
+		// flag the main thread to exit. A thread interrupt will come anyways.
+		running = false;
+		handover.close();
+		consumerThread.shutdown();
+	}
+
+	// ------------------------------------------------------------------------
+	//  The below methods are overridden in the 0.10 fetcher, which otherwise
+	//   reuses most of the 0.9 fetcher behavior
+	// ------------------------------------------------------------------------
+
+	protected void emitRecord(
+			T record,
+			KafkaTopicPartitionState<TopicPartition> partition,
+			long offset,
+			@SuppressWarnings("UnusedParameters") ConsumerRecord<?, ?> consumerRecord) throws Exception {
+
+		// the 0.9 Fetcher does not try to extract a timestamp
+		emitRecord(record, partition, offset);
+	}
+
+	/**
+	 * Gets the name of this fetcher, for thread naming and logging purposes.
+	 */
+	protected String getFetcherName() {
+		return "Kafka 0.9 Fetcher";
+	}
+
+	protected KafkaConsumerCallBridge createCallBridge() {
+		return new KafkaConsumerCallBridge();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Implement Methods of the AbstractFetcher
+	// ------------------------------------------------------------------------
+
+	@Override
+	public TopicPartition createKafkaPartitionHandle(KafkaTopicPartition partition) {
+		return new TopicPartition(partition.getTopic(), partition.getPartition());
+	}
+
+	@Override
+	public void commitInternalOffsetsToKafka(Map<KafkaTopicPartition, Long> offsets) throws Exception {
+		KafkaTopicPartitionState<TopicPartition>[] partitions = subscribedPartitions();
+		Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = new HashMap<>(partitions.length);
+
+		for (KafkaTopicPartitionState<TopicPartition> partition : partitions) {
+			Long lastProcessedOffset = offsets.get(partition.getKafkaTopicPartition());
+			if (lastProcessedOffset != null) {
+				// committed offsets through the KafkaConsumer need to be 1 more than the last processed offset.
+				// This does not affect Flink's checkpoints/saved state.
+				long offsetToCommit = lastProcessedOffset + 1;
+
+				offsetsToCommit.put(partition.getKafkaPartitionHandle(), new OffsetAndMetadata(offsetToCommit));
+				partition.setCommittedOffset(offsetToCommit);
+			}
+		}
+
+		// record the work to be committed by the main consumer thread and make sure the consumer notices that
+		consumerThread.setOffsetsToCommit(offsetsToCommit);
+	}
+}


[26/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
new file mode 100644
index 0000000..dbf95f9
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
@@ -0,0 +1,472 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis;
+
+import com.amazonaws.services.kinesis.model.Shard;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
+import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator;
+import org.apache.flink.streaming.connectors.kinesis.testutils.TestableFlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.mockito.Mockito;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Suite of FlinkKinesisConsumer tests for the methods called throughout the source life cycle.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({FlinkKinesisConsumer.class, KinesisConfigUtil.class})
+public class FlinkKinesisConsumerTest {
+
+	@Rule
+	private ExpectedException exception = ExpectedException.none();
+
+	// ----------------------------------------------------------------------
+	// FlinkKinesisConsumer.validateAwsConfiguration() tests
+	// ----------------------------------------------------------------------
+
+	@Test
+	public void testMissingAwsRegionInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("The AWS region ('" + AWSConfigConstants.AWS_REGION + "') must be set in the config.");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKey");
+		testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+		KinesisConfigUtil.validateAwsConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnrecognizableAwsRegionInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid AWS region");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(AWSConfigConstants.AWS_REGION, "wrongRegionId");
+		testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+		KinesisConfigUtil.validateAwsConfiguration(testConfig);
+	}
+
+	@Test
+	public void testCredentialProviderTypeDefaultToBasicButNoCredentialsSetInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Please set values for AWS Access Key ID ('"+ AWSConfigConstants.AWS_ACCESS_KEY_ID +"') " +
+				"and Secret Key ('" + AWSConfigConstants.AWS_SECRET_ACCESS_KEY + "') when using the BASIC AWS credential provider type.");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+
+		KinesisConfigUtil.validateAwsConfiguration(testConfig);
+	}
+
+	@Test
+	public void testCredentialProviderTypeSetToBasicButNoCredentialSetInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Please set values for AWS Access Key ID ('"+ AWSConfigConstants.AWS_ACCESS_KEY_ID +"') " +
+				"and Secret Key ('" + AWSConfigConstants.AWS_SECRET_ACCESS_KEY + "') when using the BASIC AWS credential provider type.");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER, "BASIC");
+
+		KinesisConfigUtil.validateAwsConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnrecognizableCredentialProviderTypeInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid AWS Credential Provider Type");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER, "wrongProviderType");
+		testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+		KinesisConfigUtil.validateAwsConfiguration(testConfig);
+	}
+
+	// ----------------------------------------------------------------------
+	// FlinkKinesisConsumer.validateConsumerConfiguration() tests
+	// ----------------------------------------------------------------------
+
+	@Test
+	public void testUnrecognizableStreamInitPositionTypeInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid initial position in stream");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_CREDENTIALS_PROVIDER, "BASIC");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_INITIAL_POSITION, "wrongInitPosition");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableLongForDescribeStreamBackoffBaseMillisInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for describe stream operation base backoff milliseconds");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE, "unparsableLong");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableLongForDescribeStreamBackoffMaxMillisInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for describe stream operation max backoff milliseconds");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX, "unparsableLong");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableDoubleForDescribeStreamBackoffExponentialConstantInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for describe stream operation backoff exponential constant");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, "unparsableDouble");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableIntForGetRecordsRetriesInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for maximum retry attempts for getRecords shard operation");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETRECORDS_RETRIES, "unparsableInt");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableIntForGetRecordsMaxCountInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for maximum records per getRecords shard operation");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETRECORDS_MAX, "unparsableInt");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableLongForGetRecordsBackoffBaseMillisInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for get records operation base backoff milliseconds");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_BASE, "unparsableLong");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableLongForGetRecordsBackoffMaxMillisInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for get records operation max backoff milliseconds");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_MAX, "unparsableLong");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableDoubleForGetRecordsBackoffExponentialConstantInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for get records operation backoff exponential constant");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT, "unparsableDouble");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableLongForGetRecordsIntervalMillisInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for getRecords sleep interval in milliseconds");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS, "unparsableLong");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableIntForGetShardIteratorRetriesInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for maximum retry attempts for getShardIterator shard operation");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETITERATOR_RETRIES, "unparsableInt");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableLongForGetShardIteratorBackoffBaseMillisInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for get shard iterator operation base backoff milliseconds");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_BASE, "unparsableLong");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableLongForGetShardIteratorBackoffMaxMillisInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for get shard iterator operation max backoff milliseconds");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_MAX, "unparsableLong");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableDoubleForGetShardIteratorBackoffExponentialConstantInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for get shard iterator operation backoff exponential constant");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT, "unparsableDouble");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableLongForShardDiscoveryIntervalMillisInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for shard discovery sleep interval in milliseconds");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS, "unparsableLong");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	// ----------------------------------------------------------------------
+	// FlinkKinesisConsumer.validateProducerConfiguration() tests
+	// ----------------------------------------------------------------------
+
+	@Test
+	public void testUnparsableLongForCollectionMaxCountInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for maximum number of items to pack into a PutRecords request");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ProducerConfigConstants.COLLECTION_MAX_COUNT, "unparsableLong");
+
+		KinesisConfigUtil.validateProducerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testUnparsableLongForAggregationMaxCountInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for maximum number of items to pack into an aggregated record");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ProducerConfigConstants.AGGREGATION_MAX_COUNT, "unparsableLong");
+
+		KinesisConfigUtil.validateProducerConfiguration(testConfig);
+	}
+
+	// ----------------------------------------------------------------------
+	// Tests related to state initialization
+	// ----------------------------------------------------------------------
+
+	@Test
+	public void testSnapshotStateShouldBeNullIfSourceNotOpened() throws Exception {
+		Properties config = new Properties();
+		config.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+		config.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		config.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+		FlinkKinesisConsumer<String> consumer = new FlinkKinesisConsumer<>("fakeStream", new SimpleStringSchema(), config);
+
+		assertTrue(consumer.snapshotState(123, 123) == null); //arbitrary checkpoint id and timestamp
+	}
+
+	@Test
+	public void testSnapshotStateShouldBeNullIfSourceNotRun() throws Exception {
+		Properties config = new Properties();
+		config.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+		config.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		config.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+		FlinkKinesisConsumer<String> consumer = new FlinkKinesisConsumer<>("fakeStream", new SimpleStringSchema(), config);
+		consumer.open(new Configuration()); // only opened, not run
+
+		assertTrue(consumer.snapshotState(123, 123) == null); //arbitrary checkpoint id and timestamp
+	}
+
+	// ----------------------------------------------------------------------
+	// Tests related to fetcher initialization
+	// ----------------------------------------------------------------------
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testFetcherShouldNotBeRestoringFromFailureIfNotRestoringFromCheckpoint() throws Exception {
+		KinesisDataFetcher mockedFetcher = Mockito.mock(KinesisDataFetcher.class);
+		PowerMockito.whenNew(KinesisDataFetcher.class).withAnyArguments().thenReturn(mockedFetcher);
+
+		// assume the given config is correct
+		PowerMockito.mockStatic(KinesisConfigUtil.class);
+		PowerMockito.doNothing().when(KinesisConfigUtil.class);
+
+		TestableFlinkKinesisConsumer consumer = new TestableFlinkKinesisConsumer(
+			"fakeStream", new Properties(), 10, 2);
+		consumer.open(new Configuration());
+		consumer.run(Mockito.mock(SourceFunction.SourceContext.class));
+
+		Mockito.verify(mockedFetcher).setIsRestoringFromFailure(false);
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testFetcherShouldBeCorrectlySeededIfRestoringFromCheckpoint() throws Exception {
+		KinesisDataFetcher mockedFetcher = Mockito.mock(KinesisDataFetcher.class);
+		PowerMockito.whenNew(KinesisDataFetcher.class).withAnyArguments().thenReturn(mockedFetcher);
+
+		// assume the given config is correct
+		PowerMockito.mockStatic(KinesisConfigUtil.class);
+		PowerMockito.doNothing().when(KinesisConfigUtil.class);
+
+		HashMap<KinesisStreamShard, SequenceNumber> fakeRestoredState = new HashMap<>();
+		fakeRestoredState.put(
+			new KinesisStreamShard("fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			new SequenceNumber(UUID.randomUUID().toString()));
+		fakeRestoredState.put(
+			new KinesisStreamShard("fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			new SequenceNumber(UUID.randomUUID().toString()));
+		fakeRestoredState.put(
+			new KinesisStreamShard("fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))),
+			new SequenceNumber(UUID.randomUUID().toString()));
+		fakeRestoredState.put(
+			new KinesisStreamShard("fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			new SequenceNumber(UUID.randomUUID().toString()));
+		fakeRestoredState.put(
+			new KinesisStreamShard("fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			new SequenceNumber(UUID.randomUUID().toString()));
+
+		TestableFlinkKinesisConsumer consumer = new TestableFlinkKinesisConsumer(
+			"fakeStream", new Properties(), 10, 2);
+		consumer.restoreState(fakeRestoredState);
+		consumer.open(new Configuration());
+		consumer.run(Mockito.mock(SourceFunction.SourceContext.class));
+
+		Mockito.verify(mockedFetcher).setIsRestoringFromFailure(true);
+		for (Map.Entry<KinesisStreamShard, SequenceNumber> restoredShard : fakeRestoredState.entrySet()) {
+			Mockito.verify(mockedFetcher).advanceLastDiscoveredShardOfStream(
+				restoredShard.getKey().getStreamName(), restoredShard.getKey().getShard().getShardId());
+			Mockito.verify(mockedFetcher).registerNewSubscribedShardState(
+				new KinesisStreamShardState(restoredShard.getKey(), restoredShard.getValue()));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java
new file mode 100644
index 0000000..e79f9b1
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java
@@ -0,0 +1,510 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.internals;
+
+import com.amazonaws.services.kinesis.model.Shard;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
+import org.apache.flink.streaming.connectors.kinesis.testutils.FakeKinesisBehavioursFactory;
+import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator;
+import org.apache.flink.streaming.connectors.kinesis.testutils.TestableKinesisDataFetcher;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mockito;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertTrue;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(TestableKinesisDataFetcher.class)
+public class KinesisDataFetcherTest {
+
+	@Test(expected = RuntimeException.class)
+	public void testIfNoShardsAreFoundShouldThrowException() throws Exception {
+		List<String> fakeStreams = new LinkedList<>();
+		fakeStreams.add("fakeStream1");
+		fakeStreams.add("fakeStream2");
+
+		HashMap<String, String> subscribedStreamsToLastSeenShardIdsUnderTest =
+			KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams);
+
+		TestableKinesisDataFetcher fetcher =
+			new TestableKinesisDataFetcher(
+				fakeStreams,
+				new Properties(),
+				10,
+				2,
+				new AtomicReference<Throwable>(),
+				new LinkedList<KinesisStreamShardState>(),
+				subscribedStreamsToLastSeenShardIdsUnderTest,
+				FakeKinesisBehavioursFactory.noShardsFoundForRequestedStreamsBehaviour());
+
+		fetcher.setIsRestoringFromFailure(false); // not restoring
+
+		fetcher.runFetcher(); // this should throw RuntimeException
+	}
+
+	@Test
+	public void testStreamToLastSeenShardStateIsCorrectlySetWhenNotRestoringFromFailure() throws Exception {
+		List<String> fakeStreams = new LinkedList<>();
+		fakeStreams.add("fakeStream1");
+		fakeStreams.add("fakeStream2");
+		fakeStreams.add("fakeStream3");
+		fakeStreams.add("fakeStream4");
+
+		HashMap<String, String> subscribedStreamsToLastSeenShardIdsUnderTest =
+			KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams);
+
+		Map<String,Integer> streamToShardCount = new HashMap<>();
+		Random rand = new Random();
+		for (String fakeStream : fakeStreams) {
+			streamToShardCount.put(fakeStream, rand.nextInt(5)+1);
+		}
+
+		final TestableKinesisDataFetcher fetcher =
+			new TestableKinesisDataFetcher(
+				fakeStreams,
+				new Properties(),
+				10,
+				2,
+				new AtomicReference<Throwable>(),
+				new LinkedList<KinesisStreamShardState>(),
+				subscribedStreamsToLastSeenShardIdsUnderTest,
+				FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount));
+
+		fetcher.setIsRestoringFromFailure(false);
+
+		PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class));
+		Thread runFetcherThread = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					fetcher.runFetcher();
+				} catch (Exception e) {
+					//
+				}
+			}
+		});
+		runFetcherThread.start();
+		Thread.sleep(1000); // sleep a while before closing
+		fetcher.shutdownFetcher();
+
+
+		// assert that the streams tracked in the state are identical to the subscribed streams
+		Set<String> streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet();
+		assertTrue(streamsInState.size() == fakeStreams.size());
+		assertTrue(streamsInState.containsAll(fakeStreams));
+
+		// assert that the last seen shards in state is correctly set
+		for (Map.Entry<String,String> streamToLastSeenShard : subscribedStreamsToLastSeenShardIdsUnderTest.entrySet()) {
+			assertTrue(
+				streamToLastSeenShard.getValue().equals(
+					KinesisShardIdGenerator.generateFromShardOrder(streamToShardCount.get(streamToLastSeenShard.getKey())-1)));
+		}
+	}
+
+	@Test
+	public void testStreamToLastSeenShardStateIsCorrectlySetWhenNoNewShardsSinceRestoredCheckpoint() throws Exception {
+		List<String> fakeStreams = new LinkedList<>();
+		fakeStreams.add("fakeStream1");
+		fakeStreams.add("fakeStream2");
+
+		Map<KinesisStreamShard, String> restoredStateUnderTest = new HashMap<>();
+
+		// fakeStream1 has 3 shards before restore
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))),
+			UUID.randomUUID().toString());
+
+		// fakeStream2 has 2 shards before restore
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			UUID.randomUUID().toString());
+
+		Map<String,Integer> streamToShardCount = new HashMap<>();
+		streamToShardCount.put("fakeStream1", 3); // fakeStream1 will still have 3 shards after restore
+		streamToShardCount.put("fakeStream2", 2); // fakeStream2 will still have 2 shards after restore
+
+		HashMap<String, String> subscribedStreamsToLastSeenShardIdsUnderTest =
+			KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams);
+
+		final TestableKinesisDataFetcher fetcher =
+			new TestableKinesisDataFetcher(
+				fakeStreams,
+				new Properties(),
+				10,
+				2,
+				new AtomicReference<Throwable>(),
+				new LinkedList<KinesisStreamShardState>(),
+				subscribedStreamsToLastSeenShardIdsUnderTest,
+				FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount));
+
+		for (Map.Entry<KinesisStreamShard, String> restoredState : restoredStateUnderTest.entrySet()) {
+			fetcher.advanceLastDiscoveredShardOfStream(restoredState.getKey().getStreamName(), restoredState.getKey().getShard().getShardId());
+			fetcher.registerNewSubscribedShardState(
+				new KinesisStreamShardState(restoredState.getKey(), new SequenceNumber(restoredState.getValue())));
+		}
+
+		fetcher.setIsRestoringFromFailure(true);
+
+		PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class));
+		Thread runFetcherThread = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					fetcher.runFetcher();
+				} catch (Exception e) {
+					//
+				}
+			}
+		});
+		runFetcherThread.start();
+		Thread.sleep(1000); // sleep a while before closing
+		fetcher.shutdownFetcher();
+
+		// assert that the streams tracked in the state are identical to the subscribed streams
+		Set<String> streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet();
+		assertTrue(streamsInState.size() == fakeStreams.size());
+		assertTrue(streamsInState.containsAll(fakeStreams));
+
+		// assert that the last seen shards in state is correctly set
+		for (Map.Entry<String,String> streamToLastSeenShard : subscribedStreamsToLastSeenShardIdsUnderTest.entrySet()) {
+			assertTrue(
+				streamToLastSeenShard.getValue().equals(
+					KinesisShardIdGenerator.generateFromShardOrder(streamToShardCount.get(streamToLastSeenShard.getKey())-1)));
+		}
+	}
+
+	@Test
+	public void testStreamToLastSeenShardStateIsCorrectlySetWhenNewShardsFoundSinceRestoredCheckpoint() throws Exception {
+		List<String> fakeStreams = new LinkedList<>();
+		fakeStreams.add("fakeStream1");
+		fakeStreams.add("fakeStream2");
+
+		Map<KinesisStreamShard, String> restoredStateUnderTest = new HashMap<>();
+
+		// fakeStream1 has 3 shards before restore
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))),
+			UUID.randomUUID().toString());
+
+		// fakeStream2 has 2 shards before restore
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			UUID.randomUUID().toString());
+
+		Map<String,Integer> streamToShardCount = new HashMap<>();
+		streamToShardCount.put("fakeStream1", 3+1); // fakeStream1 had 3 shards before & 1 new shard after restore
+		streamToShardCount.put("fakeStream2", 2+3); // fakeStream2 had 2 shards before & 3 new shard after restore
+
+		HashMap<String, String> subscribedStreamsToLastSeenShardIdsUnderTest =
+			KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams);
+
+		// using a non-resharded streams kinesis behaviour to represent that Kinesis is not resharded AFTER the restore
+		final TestableKinesisDataFetcher fetcher =
+			new TestableKinesisDataFetcher(
+				fakeStreams,
+				new Properties(),
+				10,
+				2,
+				new AtomicReference<Throwable>(),
+				new LinkedList<KinesisStreamShardState>(),
+				subscribedStreamsToLastSeenShardIdsUnderTest,
+				FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount));
+
+		for (Map.Entry<KinesisStreamShard, String> restoredState : restoredStateUnderTest.entrySet()) {
+			fetcher.advanceLastDiscoveredShardOfStream(restoredState.getKey().getStreamName(), restoredState.getKey().getShard().getShardId());
+			fetcher.registerNewSubscribedShardState(
+				new KinesisStreamShardState(restoredState.getKey(), new SequenceNumber(restoredState.getValue())));
+		}
+
+		fetcher.setIsRestoringFromFailure(true);
+
+		PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class));
+		Thread runFetcherThread = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					fetcher.runFetcher();
+				} catch (Exception e) {
+					//
+				}
+			}
+		});
+		runFetcherThread.start();
+		Thread.sleep(1000); // sleep a while before closing
+		fetcher.shutdownFetcher();
+
+		// assert that the streams tracked in the state are identical to the subscribed streams
+		Set<String> streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet();
+		assertTrue(streamsInState.size() == fakeStreams.size());
+		assertTrue(streamsInState.containsAll(fakeStreams));
+
+		// assert that the last seen shards in state is correctly set
+		for (Map.Entry<String,String> streamToLastSeenShard : subscribedStreamsToLastSeenShardIdsUnderTest.entrySet()) {
+			assertTrue(
+				streamToLastSeenShard.getValue().equals(
+					KinesisShardIdGenerator.generateFromShardOrder(streamToShardCount.get(streamToLastSeenShard.getKey())-1)));
+		}
+	}
+
+	@Test
+	public void testStreamToLastSeenShardStateIsCorrectlySetWhenNoNewShardsSinceRestoredCheckpointAndSomeStreamsDoNotExist() throws Exception {
+		List<String> fakeStreams = new LinkedList<>();
+		fakeStreams.add("fakeStream1");
+		fakeStreams.add("fakeStream2");
+		fakeStreams.add("fakeStream3"); // fakeStream3 will not have any shards
+		fakeStreams.add("fakeStream4"); // fakeStream4 will not have any shards
+
+		Map<KinesisStreamShard, String> restoredStateUnderTest = new HashMap<>();
+
+		// fakeStream1 has 3 shards before restore
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))),
+			UUID.randomUUID().toString());
+
+		// fakeStream2 has 2 shards before restore
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			UUID.randomUUID().toString());
+
+		Map<String,Integer> streamToShardCount = new HashMap<>();
+		streamToShardCount.put("fakeStream1", 3); // fakeStream1 has fixed 3 shards
+		streamToShardCount.put("fakeStream2", 2); // fakeStream2 has fixed 2 shards
+		streamToShardCount.put("fakeStream3", 0); // no shards can be found for fakeStream3
+		streamToShardCount.put("fakeStream4", 0); // no shards can be found for fakeStream4
+
+		HashMap<String, String> subscribedStreamsToLastSeenShardIdsUnderTest =
+			KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams);
+
+		// using a non-resharded streams kinesis behaviour to represent that Kinesis is not resharded AFTER the restore
+		final TestableKinesisDataFetcher fetcher =
+			new TestableKinesisDataFetcher(
+				fakeStreams,
+				new Properties(),
+				10,
+				2,
+				new AtomicReference<Throwable>(),
+				new LinkedList<KinesisStreamShardState>(),
+				subscribedStreamsToLastSeenShardIdsUnderTest,
+				FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount));
+
+		for (Map.Entry<KinesisStreamShard, String> restoredState : restoredStateUnderTest.entrySet()) {
+			fetcher.advanceLastDiscoveredShardOfStream(restoredState.getKey().getStreamName(), restoredState.getKey().getShard().getShardId());
+			fetcher.registerNewSubscribedShardState(
+				new KinesisStreamShardState(restoredState.getKey(), new SequenceNumber(restoredState.getValue())));
+		}
+
+		fetcher.setIsRestoringFromFailure(true);
+
+		PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class));
+		Thread runFetcherThread = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					fetcher.runFetcher();
+				} catch (Exception e) {
+					//
+				}
+			}
+		});
+		runFetcherThread.start();
+		Thread.sleep(1000); // sleep a while before closing
+		fetcher.shutdownFetcher();
+
+		// assert that the streams tracked in the state are identical to the subscribed streams
+		Set<String> streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet();
+		assertTrue(streamsInState.size() == fakeStreams.size());
+		assertTrue(streamsInState.containsAll(fakeStreams));
+
+		// assert that the last seen shards in state is correctly set
+		assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream1").equals(
+			KinesisShardIdGenerator.generateFromShardOrder(2)));
+		assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream2").equals(
+			KinesisShardIdGenerator.generateFromShardOrder(1)));
+		assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream3") == null);
+		assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream4") == null);
+	}
+
+	@Test
+	public void testStreamToLastSeenShardStateIsCorrectlySetWhenNewShardsFoundSinceRestoredCheckpointAndSomeStreamsDoNotExist() throws Exception {
+		List<String> fakeStreams = new LinkedList<>();
+		fakeStreams.add("fakeStream1");
+		fakeStreams.add("fakeStream2");
+		fakeStreams.add("fakeStream3"); // fakeStream3 will not have any shards
+		fakeStreams.add("fakeStream4"); // fakeStream4 will not have any shards
+
+		Map<KinesisStreamShard, String> restoredStateUnderTest = new HashMap<>();
+
+		// fakeStream1 has 3 shards before restore
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream1",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))),
+			UUID.randomUUID().toString());
+
+		// fakeStream2 has 2 shards before restore
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))),
+			UUID.randomUUID().toString());
+		restoredStateUnderTest.put(
+			new KinesisStreamShard(
+				"fakeStream2",
+				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))),
+			UUID.randomUUID().toString());
+
+		Map<String,Integer> streamToShardCount = new HashMap<>();
+		streamToShardCount.put("fakeStream1", 3+1); // fakeStream1 had 3 shards before & 1 new shard after restore
+		streamToShardCount.put("fakeStream2", 2+3); // fakeStream2 had 2 shards before & 2 new shard after restore
+		streamToShardCount.put("fakeStream3", 0); // no shards can be found for fakeStream3
+		streamToShardCount.put("fakeStream4", 0); // no shards can be found for fakeStream4
+
+		HashMap<String, String> subscribedStreamsToLastSeenShardIdsUnderTest =
+			KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams);
+
+		// using a non-resharded streams kinesis behaviour to represent that Kinesis is not resharded AFTER the restore
+		final TestableKinesisDataFetcher fetcher =
+			new TestableKinesisDataFetcher(
+				fakeStreams,
+				new Properties(),
+				10,
+				2,
+				new AtomicReference<Throwable>(),
+				new LinkedList<KinesisStreamShardState>(),
+				subscribedStreamsToLastSeenShardIdsUnderTest,
+				FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount));
+
+		for (Map.Entry<KinesisStreamShard, String> restoredState : restoredStateUnderTest.entrySet()) {
+			fetcher.advanceLastDiscoveredShardOfStream(restoredState.getKey().getStreamName(), restoredState.getKey().getShard().getShardId());
+			fetcher.registerNewSubscribedShardState(
+				new KinesisStreamShardState(restoredState.getKey(), new SequenceNumber(restoredState.getValue())));
+		}
+
+		fetcher.setIsRestoringFromFailure(true);
+
+		PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class));
+		Thread runFetcherThread = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					fetcher.runFetcher();
+				} catch (Exception e) {
+					//
+				}
+			}
+		});
+		runFetcherThread.start();
+		Thread.sleep(1000); // sleep a while before closing
+		fetcher.shutdownFetcher();
+
+		// assert that the streams tracked in the state are identical to the subscribed streams
+		Set<String> streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet();
+		assertTrue(streamsInState.size() == fakeStreams.size());
+		assertTrue(streamsInState.containsAll(fakeStreams));
+
+		// assert that the last seen shards in state is correctly set
+		assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream1").equals(
+			KinesisShardIdGenerator.generateFromShardOrder(3)));
+		assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream2").equals(
+			KinesisShardIdGenerator.generateFromShardOrder(4)));
+		assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream3") == null);
+		assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream4") == null);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java
new file mode 100644
index 0000000..96764a4
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.internals;
+
+import com.amazonaws.services.kinesis.model.HashKeyRange;
+import com.amazonaws.services.kinesis.model.Shard;
+import org.apache.commons.lang.StringUtils;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
+import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+import org.apache.flink.streaming.connectors.kinesis.testutils.FakeKinesisBehavioursFactory;
+import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator;
+import org.apache.flink.streaming.connectors.kinesis.testutils.TestableKinesisDataFetcher;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertTrue;
+
+public class ShardConsumerTest {
+
+	@Test
+	public void testCorrectNumOfCollectedRecordsAndUpdatedState() {
+		KinesisStreamShard fakeToBeConsumedShard = new KinesisStreamShard(
+			"fakeStream",
+			new Shard()
+				.withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))
+				.withHashKeyRange(
+					new HashKeyRange()
+						.withStartingHashKey("0")
+						.withEndingHashKey(new BigInteger(StringUtils.repeat("FF", 16), 16).toString())));
+
+		LinkedList<KinesisStreamShardState> subscribedShardsStateUnderTest = new LinkedList<>();
+		subscribedShardsStateUnderTest.add(
+			new KinesisStreamShardState(fakeToBeConsumedShard, new SequenceNumber("fakeStartingState")));
+
+		TestableKinesisDataFetcher fetcher =
+			new TestableKinesisDataFetcher(
+				Collections.singletonList("fakeStream"),
+				new Properties(),
+				10,
+				2,
+				new AtomicReference<Throwable>(),
+				subscribedShardsStateUnderTest,
+				KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(Collections.singletonList("fakeStream")),
+				Mockito.mock(KinesisProxyInterface.class));
+
+		new ShardConsumer<>(
+			fetcher,
+			0,
+			subscribedShardsStateUnderTest.get(0).getKinesisStreamShard(),
+			subscribedShardsStateUnderTest.get(0).getLastProcessedSequenceNum(),
+			FakeKinesisBehavioursFactory.totalNumOfRecordsAfterNumOfGetRecordsCalls(1000, 9)).run();
+
+		assertTrue(fetcher.getNumOfElementsCollected() == 1000);
+		assertTrue(subscribedShardsStateUnderTest.get(0).getLastProcessedSequenceNum().equals(
+			SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()));
+	}
+
+	@Test
+	public void testCorrectNumOfCollectedRecordsAndUpdatedStateWithUnexpectedExpiredIterator() {
+		KinesisStreamShard fakeToBeConsumedShard = new KinesisStreamShard(
+			"fakeStream",
+			new Shard()
+				.withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))
+				.withHashKeyRange(
+					new HashKeyRange()
+						.withStartingHashKey("0")
+						.withEndingHashKey(new BigInteger(StringUtils.repeat("FF", 16), 16).toString())));
+
+		LinkedList<KinesisStreamShardState> subscribedShardsStateUnderTest = new LinkedList<>();
+		subscribedShardsStateUnderTest.add(
+			new KinesisStreamShardState(fakeToBeConsumedShard, new SequenceNumber("fakeStartingState")));
+
+		TestableKinesisDataFetcher fetcher =
+			new TestableKinesisDataFetcher(
+				Collections.singletonList("fakeStream"),
+				new Properties(),
+				10,
+				2,
+				new AtomicReference<Throwable>(),
+				subscribedShardsStateUnderTest,
+				KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(Collections.singletonList("fakeStream")),
+				Mockito.mock(KinesisProxyInterface.class));
+
+		new ShardConsumer<>(
+			fetcher,
+			0,
+			subscribedShardsStateUnderTest.get(0).getKinesisStreamShard(),
+			subscribedShardsStateUnderTest.get(0).getLastProcessedSequenceNum(),
+			// Get a total of 1000 records with 9 getRecords() calls,
+			// and the 7th getRecords() call will encounter an unexpected expired shard iterator
+			FakeKinesisBehavioursFactory.totalNumOfRecordsAfterNumOfGetRecordsCallsWithUnexpectedExpiredIterator(1000, 9, 7)).run();
+
+		assertTrue(fetcher.getNumOfElementsCollected() == 1000);
+		assertTrue(subscribedShardsStateUnderTest.get(0).getLastProcessedSequenceNum().equals(
+			SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()));
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java
new file mode 100644
index 0000000..6e02a55
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.manualtests;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.examples.ProduceIntoKinesis;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.apache.flink.util.Collector;
+
+import java.nio.ByteBuffer;
+import java.util.Properties;
+
+/**
+ * This is a manual test for the AWS Kinesis connector in Flink.
+ *
+ * It uses:
+ *  - A custom KinesisSerializationSchema
+ *  - A custom KinesisPartitioner
+ *
+ * Invocation:
+ * --region eu-central-1 --accessKey XXXXXXXXXXXX --secretKey XXXXXXXXXXXXXXXX
+ */
+public class ManualConsumerProducerTest {
+
+	public static void main(String[] args) throws Exception {
+		ParameterTool pt = ParameterTool.fromArgs(args);
+
+		StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
+		see.setParallelism(4);
+
+		DataStream<String> simpleStringStream = see.addSource(new ProduceIntoKinesis.EventsGenerator());
+
+		Properties kinesisProducerConfig = new Properties();
+		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region"));
+		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
+		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
+
+		FlinkKinesisProducer<String> kinesis = new FlinkKinesisProducer<>(
+				new KinesisSerializationSchema<String>() {
+					@Override
+					public ByteBuffer serialize(String element) {
+						return ByteBuffer.wrap(element.getBytes());
+					}
+
+					// every 10th element goes into a different stream
+					@Override
+					public String getTargetStream(String element) {
+						if(element.split("-")[0].endsWith("0")) {
+							return "flink-test-2";
+						}
+						return null; // send to default stream
+					}
+				},
+				kinesisProducerConfig
+		);
+
+		kinesis.setFailOnError(true);
+		kinesis.setDefaultStream("test-flink");
+		kinesis.setDefaultPartition("0");
+		kinesis.setCustomPartitioner(new KinesisPartitioner<String>() {
+			@Override
+			public String getPartitionId(String element) {
+				int l = element.length();
+				return element.substring(l - 1, l);
+			}
+		});
+		simpleStringStream.addSink(kinesis);
+
+
+		// consuming topology
+		Properties consumerProps = new Properties();
+		consumerProps.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
+		consumerProps.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
+		consumerProps.setProperty(ConsumerConfigConstants.AWS_REGION, pt.getRequired("region"));
+		DataStream<String> consuming = see.addSource(new FlinkKinesisConsumer<>("test-flink", new SimpleStringSchema(), consumerProps));
+		// validate consumed records for correctness
+		consuming.flatMap(new FlatMapFunction<String, String>() {
+			@Override
+			public void flatMap(String value, Collector<String> out) throws Exception {
+				String[] parts = value.split("-");
+				try {
+					long l = Long.parseLong(parts[0]);
+					if(l < 0) {
+						throw new RuntimeException("Negative");
+					}
+				} catch(NumberFormatException nfe) {
+					throw new RuntimeException("First part of '" + value + "' is not a valid numeric type");
+				}
+				if(parts[1].length() != 12) {
+					throw new RuntimeException("Second part of '" + value + "' doesn't have 12 characters");
+				}
+			}
+		});
+		consuming.print();
+
+		see.execute();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java
new file mode 100644
index 0000000..2e452c1
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.manualtests;
+
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.model.DescribeStreamResult;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.testutils.ExactlyOnceValidatingConsumerThread;
+import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisEventsGeneratorProducerThread;
+import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * This test first starts a data generator, producing data into kinesis.
+ * Then, it starts a consuming topology, ensuring that all records up to a certain
+ * point have been seen.
+ *
+ * Invocation:
+ * --region eu-central-1 --accessKey XXXXXXXXXXXX --secretKey XXXXXXXXXXXXXXXX
+ */
+public class ManualExactlyOnceTest {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ManualExactlyOnceTest.class);
+
+	static final int TOTAL_EVENT_COUNT = 1000; // the producer writes one per 10 ms, so it runs for 10k ms = 10 seconds
+
+	public static void main(String[] args) throws Exception {
+		final ParameterTool pt = ParameterTool.fromArgs(args);
+		LOG.info("Starting exactly once test");
+
+		final String streamName = "flink-test-" + UUID.randomUUID().toString();
+		final String accessKey = pt.getRequired("accessKey");
+		final String secretKey = pt.getRequired("secretKey");
+		final String region = pt.getRequired("region");
+
+		Properties configProps = new Properties();
+		configProps.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, accessKey);
+		configProps.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, secretKey);
+		configProps.setProperty(AWSConfigConstants.AWS_REGION, region);
+		AmazonKinesisClient client = AWSUtil.createKinesisClient(configProps);
+
+		// create a stream for the test:
+		client.createStream(streamName, 1);
+
+		// wait until stream has been created
+		DescribeStreamResult status = client.describeStream(streamName);
+		LOG.info("status {}" ,status);
+		while(!status.getStreamDescription().getStreamStatus().equals("ACTIVE")) {
+			status = client.describeStream(streamName);
+			LOG.info("Status of stream {}", status);
+			Thread.sleep(1000);
+		}
+
+		final Configuration flinkConfig = new Configuration();
+		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
+		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
+
+		LocalFlinkMiniCluster flink = new LocalFlinkMiniCluster(flinkConfig, false);
+		flink.start();
+
+		final int flinkPort = flink.getLeaderRPCPort();
+
+		try {
+			final AtomicReference<Throwable> producerError = new AtomicReference<>();
+			Thread producerThread = KinesisEventsGeneratorProducerThread.create(
+				TOTAL_EVENT_COUNT, 2,
+				accessKey, secretKey, region, streamName,
+				producerError, flinkPort, flinkConfig);
+			producerThread.start();
+
+			final AtomicReference<Throwable> consumerError = new AtomicReference<>();
+			Thread consumerThread = ExactlyOnceValidatingConsumerThread.create(
+				TOTAL_EVENT_COUNT, 200, 2, 500, 500,
+				accessKey, secretKey, region, streamName,
+				consumerError, flinkPort, flinkConfig);
+			consumerThread.start();
+
+			boolean deadlinePassed = false;
+			long deadline = System.currentTimeMillis() + (1000 * 2 * 60); // wait at most for two minutes
+			// wait until both producer and consumer finishes, or an unexpected error is thrown
+			while ((consumerThread.isAlive() || producerThread.isAlive()) &&
+				(producerError.get() == null && consumerError.get() == null)) {
+				Thread.sleep(1000);
+				if (System.currentTimeMillis() >= deadline) {
+					LOG.warn("Deadline passed");
+					deadlinePassed = true;
+					break; // enough waiting
+				}
+			}
+
+			if (producerThread.isAlive()) {
+				producerThread.interrupt();
+			}
+
+			if (consumerThread.isAlive()) {
+				consumerThread.interrupt();
+			}
+
+			if (producerError.get() != null) {
+				LOG.info("+++ TEST failed! +++");
+				throw new RuntimeException("Producer failed", producerError.get());
+			}
+			if (consumerError.get() != null) {
+				LOG.info("+++ TEST failed! +++");
+				throw new RuntimeException("Consumer failed", consumerError.get());
+			}
+
+			if (!deadlinePassed) {
+				LOG.info("+++ TEST passed! +++");
+			} else {
+				LOG.info("+++ TEST failed! +++");
+			}
+
+		} finally {
+			client.deleteStream(streamName);
+			client.shutdown();
+
+			// stopping flink
+			flink.stop();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java
new file mode 100644
index 0000000..6abea2a
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java
@@ -0,0 +1,247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.manualtests;
+
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.model.DescribeStreamResult;
+import com.amazonaws.services.kinesis.model.LimitExceededException;
+import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry;
+import com.amazonaws.services.kinesis.model.PutRecordsResult;
+import com.amazonaws.services.kinesis.model.PutRecordsRequest;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.testutils.ExactlyOnceValidatingConsumerThread;
+import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator;
+import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * This test first starts a data generator, producing data into kinesis.
+ * Then, it starts a consuming topology, ensuring that all records up to a certain
+ * point have been seen. While the data generator and consuming topology is running,
+ * the kinesis stream is resharded two times.
+ *
+ * Invocation:
+ * --region eu-central-1 --accessKey XXXXXXXXXXXX --secretKey XXXXXXXXXXXXXXXX
+ */
+public class ManualExactlyOnceWithStreamReshardingTest {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ManualExactlyOnceWithStreamReshardingTest.class);
+
+	static final int TOTAL_EVENT_COUNT = 20000; // a large enough record count so we can test resharding
+
+	public static void main(String[] args) throws Exception {
+		final ParameterTool pt = ParameterTool.fromArgs(args);
+		LOG.info("Starting exactly once with stream resharding test");
+
+		final String streamName = "flink-test-" + UUID.randomUUID().toString();
+		final String accessKey = pt.getRequired("accessKey");
+		final String secretKey = pt.getRequired("secretKey");
+		final String region = pt.getRequired("region");
+
+		final Properties configProps = new Properties();
+		configProps.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, accessKey);
+		configProps.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, secretKey);
+		configProps.setProperty(ConsumerConfigConstants.AWS_REGION, region);
+		configProps.setProperty(ConsumerConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS, "0");
+		final AmazonKinesisClient client = AWSUtil.createKinesisClient(configProps);
+
+		// the stream is first created with 1 shard
+		client.createStream(streamName, 1);
+
+		// wait until stream has been created
+		DescribeStreamResult status = client.describeStream(streamName);
+		LOG.info("status {}", status);
+		while(!status.getStreamDescription().getStreamStatus().equals("ACTIVE")) {
+			status = client.describeStream(streamName);
+			LOG.info("Status of stream {}", status);
+			Thread.sleep(1000);
+		}
+
+		final Configuration flinkConfig = new Configuration();
+		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
+		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
+
+		LocalFlinkMiniCluster flink = new LocalFlinkMiniCluster(flinkConfig, false);
+		flink.start();
+
+		final int flinkPort = flink.getLeaderRPCPort();
+
+		try {
+			// we have to use a manual generator here instead of the FlinkKinesisProducer
+			// because the FlinkKinesisProducer currently has a problem where records will be resent to a shard
+			// when resharding happens; this affects the consumer exactly-once validation test and will never pass
+			final AtomicReference<Throwable> producerError = new AtomicReference<>();
+			Runnable manualGenerate = new Runnable() {
+				@Override
+				public void run() {
+					AmazonKinesisClient client = AWSUtil.createKinesisClient(configProps);
+					int count = 0;
+					final int batchSize = 30;
+					while (true) {
+						try {
+							Thread.sleep(10);
+
+							Set<PutRecordsRequestEntry> batch = new HashSet<>();
+							for (int i=count; i<count+batchSize; i++) {
+								if (i >= TOTAL_EVENT_COUNT) {
+									break;
+								}
+								batch.add(
+									new PutRecordsRequestEntry()
+										.withData(ByteBuffer.wrap(((i) + "-" + RandomStringUtils.randomAlphabetic(12)).getBytes()))
+										.withPartitionKey(UUID.randomUUID().toString()));
+							}
+							count += batchSize;
+
+							PutRecordsResult result = client.putRecords(new PutRecordsRequest().withStreamName(streamName).withRecords(batch));
+
+							// the putRecords() operation may have failing records; to keep this test simple
+							// instead of retrying on failed records, we simply pass on a runtime exception
+							// and let this test fail
+							if (result.getFailedRecordCount() > 0) {
+								producerError.set(new RuntimeException("The producer has failed records in one of the put batch attempts."));
+								break;
+							}
+
+							if (count >= TOTAL_EVENT_COUNT) {
+								break;
+							}
+						} catch (Exception e) {
+							producerError.set(e);
+						}
+					}
+				}
+			};
+			Thread producerThread = new Thread(manualGenerate);
+			producerThread.start();
+
+			final AtomicReference<Throwable> consumerError = new AtomicReference<>();
+			Thread consumerThread = ExactlyOnceValidatingConsumerThread.create(
+				TOTAL_EVENT_COUNT, 10000, 2, 500, 500,
+				accessKey, secretKey, region, streamName,
+				consumerError, flinkPort, flinkConfig);
+			consumerThread.start();
+
+			// reshard the Kinesis stream while the producer / and consumers are running
+			Runnable splitShard = new Runnable() {
+				@Override
+				public void run() {
+					try {
+						// first, split shard in the middle of the hash range
+						Thread.sleep(5000);
+						LOG.info("Splitting shard ...");
+						client.splitShard(
+							streamName,
+							KinesisShardIdGenerator.generateFromShardOrder(0),
+							"170141183460469231731687303715884105727");
+
+						// wait until the split shard operation finishes updating ...
+						DescribeStreamResult status;
+						Random rand = new Random();
+						do {
+							status = null;
+							while (status == null) {
+								// retry until we get status
+								try {
+									status = client.describeStream(streamName);
+								} catch (LimitExceededException lee) {
+									LOG.warn("LimitExceededException while describing stream ... retrying ...");
+									Thread.sleep(rand.nextInt(1200));
+								}
+							}
+						} while (!status.getStreamDescription().getStreamStatus().equals("ACTIVE"));
+
+						// then merge again
+						Thread.sleep(7000);
+						LOG.info("Merging shards ...");
+						client.mergeShards(
+							streamName,
+							KinesisShardIdGenerator.generateFromShardOrder(1),
+							KinesisShardIdGenerator.generateFromShardOrder(2));
+					} catch (InterruptedException iex) {
+						//
+					}
+				}
+			};
+			Thread splitShardThread = new Thread(splitShard);
+			splitShardThread.start();
+
+			boolean deadlinePassed = false;
+			long deadline = System.currentTimeMillis() + (1000 * 5 * 60); // wait at most for five minutes
+			// wait until both producer and consumer finishes, or an unexpected error is thrown
+			while ((consumerThread.isAlive() || producerThread.isAlive()) &&
+				(producerError.get() == null && consumerError.get() == null)) {
+				Thread.sleep(1000);
+				if (System.currentTimeMillis() >= deadline) {
+					LOG.warn("Deadline passed");
+					deadlinePassed = true;
+					break; // enough waiting
+				}
+			}
+
+			if (producerThread.isAlive()) {
+				producerThread.interrupt();
+			}
+
+			if (consumerThread.isAlive()) {
+				consumerThread.interrupt();
+			}
+
+			if (producerError.get() != null) {
+				LOG.info("+++ TEST failed! +++");
+				throw new RuntimeException("Producer failed", producerError.get());
+
+			}
+
+			if (consumerError.get() != null) {
+				LOG.info("+++ TEST failed! +++");
+				throw new RuntimeException("Consumer failed", consumerError.get());
+			}
+
+			if (!deadlinePassed) {
+				LOG.info("+++ TEST passed! +++");
+			} else {
+				LOG.info("+++ TEST failed! +++");
+			}
+
+		} finally {
+			client.deleteStream(streamName);
+			client.shutdown();
+
+			// stopping flink
+			flink.stop();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java
new file mode 100644
index 0000000..35e9ef6
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.manualtests;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.examples.ProduceIntoKinesis;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema;
+
+import java.nio.ByteBuffer;
+import java.util.Properties;
+
+/**
+ * This is a manual test for the AWS Kinesis connector in Flink.
+ *
+ * It uses:
+ *  - A custom KinesisSerializationSchema
+ *  - A custom KinesisPartitioner
+ *
+ *  The streams "test-flink" and "flink-test-2" must exist.
+ *
+ * Invocation:
+ * --region eu-central-1 --accessKey XXXXXXXXXXXX --secretKey XXXXXXXXXXXXXXXX
+ */
+public class ManualProducerTest {
+
+	public static void main(String[] args) throws Exception {
+		ParameterTool pt = ParameterTool.fromArgs(args);
+
+		StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
+		see.setParallelism(4);
+
+		DataStream<String> simpleStringStream = see.addSource(new ProduceIntoKinesis.EventsGenerator());
+
+		Properties kinesisProducerConfig = new Properties();
+		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region"));
+		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
+		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
+
+		FlinkKinesisProducer<String> kinesis = new FlinkKinesisProducer<>(
+				new KinesisSerializationSchema<String>() {
+					@Override
+					public ByteBuffer serialize(String element) {
+						return ByteBuffer.wrap(element.getBytes());
+					}
+
+					// every 10th element goes into a different stream
+					@Override
+					public String getTargetStream(String element) {
+						if(element.split("-")[0].endsWith("0")) {
+							return "flink-test-2";
+						}
+						return null; // send to default stream
+					}
+				},
+				kinesisProducerConfig
+		);
+
+		kinesis.setFailOnError(true);
+		kinesis.setDefaultStream("test-flink");
+		kinesis.setDefaultPartition("0");
+		kinesis.setCustomPartitioner(new KinesisPartitioner<String>() {
+			@Override
+			public String getPartitionId(String element) {
+				int l = element.length();
+				return element.substring(l - 1, l);
+			}
+		});
+		simpleStringStream.addSink(kinesis);
+
+		see.execute();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
new file mode 100644
index 0000000..157964c
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.testutils;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.apache.flink.test.util.SuccessException;
+import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.BitSet;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.test.util.TestUtils.tryExecute;
+
+/**
+ * A thread that runs a topology with the FlinkKinesisConsumer as source, followed by two flat map
+ * functions, one that performs artificial failures and another that validates exactly-once guarantee
+ */
+public class ExactlyOnceValidatingConsumerThread {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ExactlyOnceValidatingConsumerThread.class);
+
+	public static Thread create(final int totalEventCount,
+								final int failAtRecordCount,
+								final int parallelism,
+								final int checkpointInterval,
+								final long restartDelay,
+								final String awsAccessKey,
+								final String awsSecretKey,
+								final String awsRegion,
+								final String kinesisStreamName,
+								final AtomicReference<Throwable> errorHandler,
+								final int flinkPort,
+								final Configuration flinkConfig) {
+		Runnable exactlyOnceValidationConsumer = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort, flinkConfig);
+					see.setParallelism(parallelism);
+					see.enableCheckpointing(checkpointInterval);
+					// we restart two times
+					see.setRestartStrategy(RestartStrategies.fixedDelayRestart(2, restartDelay));
+
+					// consuming topology
+					Properties consumerProps = new Properties();
+					consumerProps.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, awsAccessKey);
+					consumerProps.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, awsSecretKey);
+					consumerProps.setProperty(ConsumerConfigConstants.AWS_REGION, awsRegion);
+					// start reading from beginning
+					consumerProps.setProperty(ConsumerConfigConstants.STREAM_INITIAL_POSITION, ConsumerConfigConstants.InitialPosition.TRIM_HORIZON.name());
+					DataStream<String> consuming = see.addSource(new FlinkKinesisConsumer<>(kinesisStreamName, new SimpleStringSchema(), consumerProps));
+					consuming
+						.flatMap(new ArtificialFailOnceFlatMapper(failAtRecordCount))
+						// validate consumed records for correctness (use only 1 instance to validate all consumed records)
+						.flatMap(new ExactlyOnceValidatingMapper(totalEventCount)).setParallelism(1);
+
+					LOG.info("Starting consuming topology");
+					tryExecute(see, "Consuming topo");
+					LOG.info("Consuming topo finished");
+				} catch (Exception e) {
+					LOG.warn("Error while running consuming topology", e);
+					errorHandler.set(e);
+				}
+			}
+		};
+
+		return new Thread(exactlyOnceValidationConsumer);
+	}
+
+	private static class ExactlyOnceValidatingMapper implements FlatMapFunction<String,String>, Checkpointed<BitSet> {
+
+		private static final Logger LOG = LoggerFactory.getLogger(ExactlyOnceValidatingMapper.class);
+
+		private final int totalEventCount;
+		private BitSet validator;
+
+		public ExactlyOnceValidatingMapper(int totalEventCount) {
+			this.totalEventCount = totalEventCount;
+			this.validator = new BitSet(totalEventCount);
+		}
+
+		@Override
+		public void flatMap(String value, Collector<String> out) throws Exception {
+			LOG.info("Consumed {}", value);
+
+			int id = Integer.parseInt(value.split("-")[0]);
+			if(validator.get(id)) {
+				throw new RuntimeException("Saw id " + id +" twice!");
+			}
+			validator.set(id);
+			if(id > totalEventCount-1) {
+				throw new RuntimeException("Out of bounds ID observed");
+			}
+
+			if(validator.nextClearBit(0) == totalEventCount) {
+				throw new SuccessException();
+			}
+		}
+
+		@Override
+		public BitSet snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return validator;
+		}
+
+		@Override
+		public void restoreState(BitSet state) throws Exception {
+			this.validator = state;
+		}
+	}
+
+	private static class ArtificialFailOnceFlatMapper extends RichFlatMapFunction<String,String> {
+		int count = 0;
+
+		private final int failAtRecordCount;
+
+		public ArtificialFailOnceFlatMapper(int failAtRecordCount) {
+			this.failAtRecordCount = failAtRecordCount;
+		}
+
+		@Override
+		public void flatMap(String value, Collector<String> out) throws Exception {
+			if (count++ >= failAtRecordCount && getRuntimeContext().getAttemptNumber() == 0) {
+				throw new RuntimeException("Artificial failure. Restart please.");
+			}
+			out.collect(value);
+		}
+	}
+}


[18/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSink.java b/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSink.java
deleted file mode 100644
index a3d002e..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSink.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.cassandra;
-
-import com.datastax.driver.core.BoundStatement;
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
-import com.datastax.driver.core.Session;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
-import org.apache.flink.streaming.runtime.operators.GenericWriteAheadSink;
-
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Sink that emits its input elements into a Cassandra database. This sink stores incoming records within a
- * {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only commits them to cassandra
- * if a checkpoint is completed.
- *
- * @param <IN> Type of the elements emitted by this sink
- */
-public class CassandraTupleWriteAheadSink<IN extends Tuple> extends GenericWriteAheadSink<IN> {
-	private static final long serialVersionUID = 1L;
-
-	protected transient Cluster cluster;
-	protected transient Session session;
-
-	private final String insertQuery;
-	private transient PreparedStatement preparedStatement;
-
-	private ClusterBuilder builder;
-
-	private transient Object[] fields;
-
-	protected CassandraTupleWriteAheadSink(String insertQuery, TypeSerializer<IN> serializer, ClusterBuilder builder, CheckpointCommitter committer) throws Exception {
-		super(committer, serializer, UUID.randomUUID().toString().replace("-", "_"));
-		this.insertQuery = insertQuery;
-		this.builder = builder;
-		ClosureCleaner.clean(builder, true);
-	}
-
-	public void open() throws Exception {
-		super.open();
-		if (!getRuntimeContext().isCheckpointingEnabled()) {
-			throw new IllegalStateException("The write-ahead log requires checkpointing to be enabled.");
-		}
-		cluster = builder.getCluster();
-		session = cluster.connect();
-		preparedStatement = session.prepare(insertQuery);
-
-		fields = new Object[((TupleSerializer<IN>) serializer).getArity()];
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		try {
-			if (session != null) {
-				session.close();
-			}
-		} catch (Exception e) {
-			LOG.error("Error while closing session.", e);
-		}
-		try {
-			if (cluster != null) {
-				cluster.close();
-			}
-		} catch (Exception e) {
-			LOG.error("Error while closing cluster.", e);
-		}
-	}
-
-	@Override
-	protected boolean sendValues(Iterable<IN> values, long timestamp) throws Exception {
-		final AtomicInteger updatesCount = new AtomicInteger(0);
-		final AtomicInteger updatesConfirmed = new AtomicInteger(0);
-
-		final AtomicReference<Throwable> exception = new AtomicReference<>();
-
-		FutureCallback<ResultSet> callback = new FutureCallback<ResultSet>() {
-			@Override
-			public void onSuccess(ResultSet resultSet) {
-				updatesConfirmed.incrementAndGet();
-				if (updatesCount.get() > 0) { // only set if all updates have been sent
-					if (updatesCount.get() == updatesConfirmed.get()) {
-						synchronized (updatesConfirmed) {
-							updatesConfirmed.notifyAll();
-						}
-					}
-				}
-			}
-
-			@Override
-			public void onFailure(Throwable throwable) {
-				if (exception.compareAndSet(null, throwable)) {
-					LOG.error("Error while sending value.", throwable);
-					synchronized (updatesConfirmed) {
-						updatesConfirmed.notifyAll();
-					}
-				}
-			}
-		};
-
-		//set values for prepared statement
-		int updatesSent = 0;
-		for (IN value : values) {
-			for (int x = 0; x < value.getArity(); x++) {
-				fields[x] = value.getField(x);
-			}
-			//insert values and send to cassandra
-			BoundStatement s = preparedStatement.bind(fields);
-			s.setDefaultTimestamp(timestamp);
-			ResultSetFuture result = session.executeAsync(s);
-			updatesSent++;
-			if (result != null) {
-				//add callback to detect errors
-				Futures.addCallback(result, callback);
-			}
-		}
-		updatesCount.set(updatesSent);
-
-		synchronized (updatesConfirmed) {
-			while (exception.get() == null && updatesSent != updatesConfirmed.get()) {
-				updatesConfirmed.wait();
-			}
-		}
-
-		if (exception.get() != null) {
-			LOG.warn("Sending a value failed.", exception.get());
-			return false;
-		} else {
-			return true;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/ClusterBuilder.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/ClusterBuilder.java b/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/ClusterBuilder.java
deleted file mode 100644
index 9fd3b4e..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/ClusterBuilder.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.cassandra;
-
-import com.datastax.driver.core.Cluster;
-
-import java.io.Serializable;
-
-/**
- * This class is used to configure a {@link com.datastax.driver.core.Cluster} after deployment.
- * The cluster represents the connection that will be established to Cassandra.
- */
-public abstract class ClusterBuilder implements Serializable {
-
-	public Cluster getCluster() {
-		return buildCluster(Cluster.builder());
-	}
-
-	/**
-	 * Configures the connection to Cassandra.
-	 * The configuration is done by calling methods on the builder object
-	 * and finalizing the configuration with build().
-	 *
-	 * @param builder connection builder
-	 * @return configured connection
-	 */
-	protected abstract Cluster buildCluster(Cluster.Builder builder);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java
deleted file mode 100644
index e66b8b3..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.batch.connectors.cassandra.example;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.Cluster.Builder;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.batch.connectors.cassandra.CassandraInputFormat;
-import org.apache.flink.batch.connectors.cassandra.CassandraOutputFormat;
-import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
-
-import java.util.ArrayList;
-
-/**
- * This is an example showing the to use the Cassandra Input-/OutputFormats in the Batch API.
- * 
- * The example assumes that a table exists in a local cassandra database, according to the following query: 
- * CREATE TABLE test.batches (number int, strings text, PRIMARY KEY(number, strings));
- */
-public class BatchExample {
-	private static final String INSERT_QUERY = "INSERT INTO test.batches (number, strings) VALUES (?,?);";
-	private static final String SELECT_QUERY = "SELECT number, strings FROM test.batches;";
-
-	/*
-	 *	table script: "CREATE TABLE test.batches (number int, strings text, PRIMARY KEY(number, strings));"
-	 */
-	public static void main(String[] args) throws Exception {
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		ArrayList<Tuple2<Integer, String>> collection = new ArrayList<>(20);
-		for (int i = 0; i < 20; i++) {
-			collection.add(new Tuple2<>(i, "string " + i));
-		}
-
-		DataSet<Tuple2<Integer, String>> dataSet = env.fromCollection(collection);
-
-		dataSet.output(new CassandraOutputFormat<Tuple2<Integer, String>>(INSERT_QUERY, new ClusterBuilder() {
-			@Override
-			protected Cluster buildCluster(Builder builder) {
-				return builder.addContactPoints("127.0.0.1").build();
-			}
-		}));
-
-		env.execute("Write");
-
-		DataSet<Tuple2<Integer, String>> inputDS = env
-			.createInput(new CassandraInputFormat<Tuple2<Integer, String>>(SELECT_QUERY, new ClusterBuilder() {
-				@Override
-				protected Cluster buildCluster(Builder builder) {
-					return builder.addContactPoints("127.0.0.1").build();
-				}
-			}), TupleTypeInfo.of(new TypeHint<Tuple2<Integer, String>>() {
-			}));
-
-		inputDS.print();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
deleted file mode 100644
index 2bb6fd1..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
+++ /dev/null
@@ -1,440 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cassandra;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.ConsistencyLevel;
-import com.datastax.driver.core.QueryOptions;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Row;
-import com.datastax.driver.core.Session;
-
-import org.apache.cassandra.service.CassandraDaemon;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.batch.connectors.cassandra.CassandraInputFormat;
-import org.apache.flink.batch.connectors.cassandra.CassandraOutputFormat;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-import org.apache.flink.runtime.testutils.CommonTestUtils;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.runtime.operators.WriteAheadSinkTestBase;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.test.util.TestEnvironment;
-
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import org.powermock.core.classloader.annotations.PowerMockIgnore;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Scanner;
-import java.util.UUID;
-
-import static org.junit.Assert.*;
-
-@SuppressWarnings("serial")
-public class CassandraConnectorITCase extends WriteAheadSinkTestBase<Tuple3<String, Integer, Integer>, CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>>> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(CassandraConnectorITCase.class);
-	private static File tmpDir;
-
-	private static final boolean EMBEDDED = true;
-
-	private static EmbeddedCassandraService cassandra;
-
-	private static ClusterBuilder builder = new ClusterBuilder() {
-		@Override
-		protected Cluster buildCluster(Cluster.Builder builder) {
-			return builder
-				.addContactPoint("127.0.0.1")
-				.withQueryOptions(new QueryOptions().setConsistencyLevel(ConsistencyLevel.ONE).setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL))
-				.withoutJMXReporting()
-				.withoutMetrics().build();
-		}
-	};
-
-	private static Cluster cluster;
-	private static Session session;
-
-	private static final String CREATE_KEYSPACE_QUERY = "CREATE KEYSPACE flink WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
-	private static final String DROP_KEYSPACE_QUERY = "DROP KEYSPACE flink;";
-	private static final String CREATE_TABLE_QUERY = "CREATE TABLE flink.test (id text PRIMARY KEY, counter int, batch_id int);";
-	private static final String CLEAR_TABLE_QUERY = "TRUNCATE flink.test;";
-	private static final String INSERT_DATA_QUERY = "INSERT INTO flink.test (id, counter, batch_id) VALUES (?, ?, ?)";
-	private static final String SELECT_DATA_QUERY = "SELECT * FROM flink.test;";
-
-	private static final ArrayList<Tuple3<String, Integer, Integer>> collection = new ArrayList<>(20);
-
-	static {
-		for (int i = 0; i < 20; i++) {
-			collection.add(new Tuple3<>(UUID.randomUUID().toString(), i, 0));
-		}
-	}
-
-	private static class EmbeddedCassandraService {
-		CassandraDaemon cassandraDaemon;
-
-		public void start() throws IOException {
-			this.cassandraDaemon = new CassandraDaemon();
-			this.cassandraDaemon.init(null);
-			this.cassandraDaemon.start();
-		}
-
-		public void stop() {
-			this.cassandraDaemon.stop();
-		}
-	}
-
-	private static LocalFlinkMiniCluster flinkCluster;
-
-	// ------------------------------------------------------------------------
-	//  Cluster Setup (Cassandra & Flink)
-	// ------------------------------------------------------------------------
-
-	@BeforeClass
-	public static void startCassandra() throws IOException {
-
-		// check if we should run this test, current Cassandra version requires Java >= 1.8
-		org.apache.flink.core.testutils.CommonTestUtils.assumeJava8();
-
-		// generate temporary files
-		tmpDir = CommonTestUtils.createTempDirectory();
-		ClassLoader classLoader = CassandraConnectorITCase.class.getClassLoader();
-		File file = new File(classLoader.getResource("cassandra.yaml").getFile());
-		File tmp = new File(tmpDir.getAbsolutePath() + File.separator + "cassandra.yaml");
-		
-		assertTrue(tmp.createNewFile());
-
-		try (
-			BufferedWriter b = new BufferedWriter(new FileWriter(tmp));
-
-			//copy cassandra.yaml; inject absolute paths into cassandra.yaml
-			Scanner scanner = new Scanner(file);
-		) {
-			while (scanner.hasNextLine()) {
-				String line = scanner.nextLine();
-				line = line.replace("$PATH", "'" + tmp.getParentFile());
-				b.write(line + "\n");
-				b.flush();
-			}
-		}
-
-
-		// Tell cassandra where the configuration files are.
-		// Use the test configuration file.
-		System.setProperty("cassandra.config", tmp.getAbsoluteFile().toURI().toString());
-
-		if (EMBEDDED) {
-			cassandra = new EmbeddedCassandraService();
-			cassandra.start();
-		}
-
-		try {
-			Thread.sleep(1000 * 10);
-		} catch (InterruptedException e) { //give cassandra a few seconds to start up
-		}
-
-		cluster = builder.getCluster();
-		session = cluster.connect();
-
-		session.execute(CREATE_KEYSPACE_QUERY);
-		session.execute(CREATE_TABLE_QUERY);
-	}
-
-	@BeforeClass
-	public static void startFlink() throws Exception {
-		Configuration config = new Configuration();
-		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
-
-		flinkCluster = new LocalFlinkMiniCluster(config);
-		flinkCluster.start();
-	}
-
-	@AfterClass
-	public static void stopFlink() {
-		if (flinkCluster != null) {
-			flinkCluster.stop();
-			flinkCluster = null;
-		}
-	}
-
-	@AfterClass
-	public static void closeCassandra() {
-		if (session != null) {
-			session.executeAsync(DROP_KEYSPACE_QUERY);
-			session.close();
-		}
-
-		if (cluster != null) {
-			cluster.close();
-		}
-
-		if (cassandra != null) {
-			cassandra.stop();
-		}
-
-		if (tmpDir != null) {
-			//noinspection ResultOfMethodCallIgnored
-			tmpDir.delete();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Test preparation & cleanup
-	// ------------------------------------------------------------------------
-
-	@Before
-	public void initializeExecutionEnvironment() {
-		TestStreamEnvironment.setAsContext(flinkCluster, 4);
-		new TestEnvironment(flinkCluster, 4, false).setAsContext();
-	}
-
-	@After
-	public void deleteSchema() throws Exception {
-		session.executeAsync(CLEAR_TABLE_QUERY);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Exactly-once Tests
-	// ------------------------------------------------------------------------
-
-	@Override
-	protected CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>> createSink() throws Exception {
-		return new CassandraTupleWriteAheadSink<>(
-			INSERT_DATA_QUERY,
-			TypeExtractor.getForObject(new Tuple3<>("", 0, 0)).createSerializer(new ExecutionConfig()),
-			builder,
-			new CassandraCommitter(builder));
-	}
-
-	@Override
-	protected TupleTypeInfo<Tuple3<String, Integer, Integer>> createTypeInfo() {
-		return TupleTypeInfo.getBasicTupleTypeInfo(String.class, Integer.class, Integer.class);
-	}
-
-	@Override
-	protected Tuple3<String, Integer, Integer> generateValue(int counter, int checkpointID) {
-		return new Tuple3<>(UUID.randomUUID().toString(), counter, checkpointID);
-	}
-
-	@Override
-	protected void verifyResultsIdealCircumstances(
-		OneInputStreamOperatorTestHarness<Tuple3<String, Integer, Integer>, Tuple3<String, Integer, Integer>> harness,
-		CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>> sink) {
-
-		ResultSet result = session.execute(SELECT_DATA_QUERY);
-		ArrayList<Integer> list = new ArrayList<>();
-		for (int x = 1; x <= 60; x++) {
-			list.add(x);
-		}
-
-		for (Row s : result) {
-			list.remove(new Integer(s.getInt("counter")));
-		}
-		Assert.assertTrue("The following ID's were not found in the ResultSet: " + list.toString(), list.isEmpty());
-	}
-
-	@Override
-	protected void verifyResultsDataPersistenceUponMissedNotify(
-		OneInputStreamOperatorTestHarness<Tuple3<String, Integer, Integer>, Tuple3<String, Integer, Integer>> harness,
-		CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>> sink) {
-
-		ResultSet result = session.execute(SELECT_DATA_QUERY);
-		ArrayList<Integer> list = new ArrayList<>();
-		for (int x = 1; x <= 60; x++) {
-			list.add(x);
-		}
-
-		for (Row s : result) {
-			list.remove(new Integer(s.getInt("counter")));
-		}
-		Assert.assertTrue("The following ID's were not found in the ResultSet: " + list.toString(), list.isEmpty());
-	}
-
-	@Override
-	protected void verifyResultsDataDiscardingUponRestore(
-		OneInputStreamOperatorTestHarness<Tuple3<String, Integer, Integer>, Tuple3<String, Integer, Integer>> harness,
-		CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>> sink) {
-
-		ResultSet result = session.execute(SELECT_DATA_QUERY);
-		ArrayList<Integer> list = new ArrayList<>();
-		for (int x = 1; x <= 20; x++) {
-			list.add(x);
-		}
-		for (int x = 41; x <= 60; x++) {
-			list.add(x);
-		}
-
-		for (Row s : result) {
-			list.remove(new Integer(s.getInt("counter")));
-		}
-		Assert.assertTrue("The following ID's were not found in the ResultSet: " + list.toString(), list.isEmpty());
-	}
-
-	@Test
-	public void testCassandraCommitter() throws Exception {
-		CassandraCommitter cc1 = new CassandraCommitter(builder);
-		cc1.setJobId("job");
-		cc1.setOperatorId("operator");
-
-		CassandraCommitter cc2 = new CassandraCommitter(builder);
-		cc2.setJobId("job");
-		cc2.setOperatorId("operator");
-
-		CassandraCommitter cc3 = new CassandraCommitter(builder);
-		cc3.setJobId("job");
-		cc3.setOperatorId("operator1");
-
-		cc1.createResource();
-
-		cc1.open();
-		cc2.open();
-		cc3.open();
-
-		Assert.assertFalse(cc1.isCheckpointCommitted(0, 1));
-		Assert.assertFalse(cc2.isCheckpointCommitted(1, 1));
-		Assert.assertFalse(cc3.isCheckpointCommitted(0, 1));
-
-		cc1.commitCheckpoint(0, 1);
-		Assert.assertTrue(cc1.isCheckpointCommitted(0, 1));
-		//verify that other sub-tasks aren't affected
-		Assert.assertFalse(cc2.isCheckpointCommitted(1, 1));
-		//verify that other tasks aren't affected
-		Assert.assertFalse(cc3.isCheckpointCommitted(0, 1));
-
-		Assert.assertFalse(cc1.isCheckpointCommitted(0, 2));
-
-		cc1.close();
-		cc2.close();
-		cc3.close();
-
-		cc1 = new CassandraCommitter(builder);
-		cc1.setJobId("job");
-		cc1.setOperatorId("operator");
-
-		cc1.open();
-
-		//verify that checkpoint data is not destroyed within open/close and not reliant on internally cached data
-		Assert.assertTrue(cc1.isCheckpointCommitted(0, 1));
-		Assert.assertFalse(cc1.isCheckpointCommitted(0, 2));
-
-		cc1.close();
-	}
-
-	// ------------------------------------------------------------------------
-	//  At-least-once Tests
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testCassandraTupleAtLeastOnceSink() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataStream<Tuple3<String, Integer, Integer>> source = env.fromCollection(collection);
-		source.addSink(new CassandraTupleSink<Tuple3<String, Integer, Integer>>(INSERT_DATA_QUERY, builder));
-
-		env.execute();
-
-		ResultSet rs = session.execute(SELECT_DATA_QUERY);
-		Assert.assertEquals(20, rs.all().size());
-	}
-
-	@Test
-	public void testCassandraPojoAtLeastOnceSink() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataStreamSource<Pojo> source = env
-			.addSource(new SourceFunction<Pojo>() {
-
-				private boolean running = true;
-				private volatile int cnt = 0;
-
-				@Override
-				public void run(SourceContext<Pojo> ctx) throws Exception {
-					while (running) {
-						ctx.collect(new Pojo(UUID.randomUUID().toString(), cnt, 0));
-						cnt++;
-						if (cnt == 20) {
-							cancel();
-						}
-					}
-				}
-
-				@Override
-				public void cancel() {
-					running = false;
-				}
-			});
-
-		source.addSink(new CassandraPojoSink<>(Pojo.class, builder));
-
-		env.execute();
-
-		ResultSet rs = session.execute(SELECT_DATA_QUERY);
-		Assert.assertEquals(20, rs.all().size());
-	}
-
-	@Test
-	public void testCassandraBatchFormats() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple3<String, Integer, Integer>> dataSet = env.fromCollection(collection);
-		dataSet.output(new CassandraOutputFormat<Tuple3<String, Integer, Integer>>(INSERT_DATA_QUERY, builder));
-
-		env.execute("Write data");
-
-		DataSet<Tuple3<String, Integer, Integer>> inputDS = env.createInput(
-			new CassandraInputFormat<Tuple3<String, Integer, Integer>>(SELECT_DATA_QUERY, builder),
-			TypeInformation.of(new TypeHint<Tuple3<String, Integer, Integer>>(){}));
-
-
-		long count = inputDS.count();
-		Assert.assertEquals(count, 20L);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSinkTest.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSinkTest.java
deleted file mode 100644
index 847d1a0..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSinkTest.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.cassandra;
-
-import com.datastax.driver.core.BoundStatement;
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSetFuture;
-import com.datastax.driver.core.Session;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.java.tuple.Tuple0;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.Collections;
-import java.util.concurrent.Executor;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.assertFalse;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.powermock.api.mockito.PowerMockito.doAnswer;
-import static org.powermock.api.mockito.PowerMockito.mock;
-import static org.powermock.api.mockito.PowerMockito.when;
-
-public class CassandraTupleWriteAheadSinkTest {
-
-	@Test(timeout=20000)
-	public void testAckLoopExitOnException() throws Exception {
-		final AtomicReference<Runnable> runnableFuture = new AtomicReference<>();
-
-		final ClusterBuilder clusterBuilder = new ClusterBuilder() {
-			private static final long serialVersionUID = 4624400760492936756L;
-
-			@Override
-			protected Cluster buildCluster(Cluster.Builder builder) {
-				try {
-					BoundStatement boundStatement = mock(BoundStatement.class);
-					when(boundStatement.setDefaultTimestamp(any(long.class))).thenReturn(boundStatement);
-
-					PreparedStatement preparedStatement = mock(PreparedStatement.class);
-					when(preparedStatement.bind(Matchers.anyVararg())).thenReturn(boundStatement);
-
-					ResultSetFuture future = mock(ResultSetFuture.class);
-					when(future.get()).thenThrow(new RuntimeException("Expected exception."));
-
-					doAnswer(new Answer<Void>() {
-						@Override
-						public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
-							synchronized (runnableFuture) {
-								runnableFuture.set((((Runnable) invocationOnMock.getArguments()[0])));
-								runnableFuture.notifyAll();
-							}
-							return null;
-						}
-					}).when(future).addListener(any(Runnable.class), any(Executor.class));
-
-					Session session = mock(Session.class);
-					when(session.prepare(anyString())).thenReturn(preparedStatement);
-					when(session.executeAsync(any(BoundStatement.class))).thenReturn(future);
-
-					Cluster cluster = mock(Cluster.class);
-					when(cluster.connect()).thenReturn(session);
-					return cluster;
-				} catch (Exception e) {
-					throw new RuntimeException(e);
-				}
-			}
-		};
-
-		// Our asynchronous executor thread
-		new Thread(new Runnable() {
-			@Override
-			public void run() {
-				synchronized (runnableFuture) {
-					while (runnableFuture.get() == null) {
-						try {
-							runnableFuture.wait();
-						} catch (InterruptedException e) {
-							// ignore interrupts
-						}
-					}
-				}
-				runnableFuture.get().run();
-			}
-		}).start();
-
-		CheckpointCommitter cc = mock(CheckpointCommitter.class);
-		final CassandraTupleWriteAheadSink<Tuple0> sink = new CassandraTupleWriteAheadSink<>(
-			"abc",
-			TupleTypeInfo.of(Tuple0.class).createSerializer(new ExecutionConfig()),
-			clusterBuilder,
-			cc
-		);
-
-		OneInputStreamOperatorTestHarness<Tuple0, Tuple0> harness = new OneInputStreamOperatorTestHarness(sink);
-		harness.getEnvironment().getTaskConfiguration().setBoolean("checkpointing", true);
-
-		harness.setup();
-		sink.open();
-
-		// we should leave the loop and return false since we've seen an exception
-		assertFalse(sink.sendValues(Collections.singleton(new Tuple0()), 0L));
-
-		sink.close();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Pojo.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Pojo.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Pojo.java
deleted file mode 100644
index 9b331d6..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Pojo.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.cassandra;
-
-import com.datastax.driver.mapping.annotations.Column;
-import com.datastax.driver.mapping.annotations.Table;
-
-import java.io.Serializable;
-
-@Table(keyspace = "flink", name = "test")
-public class Pojo implements Serializable {
-
-	private static final long serialVersionUID = 1038054554690916991L;
-
-	@Column(name = "id")
-	private String id;
-	@Column(name = "counter")
-	private int counter;
-	@Column(name = "batch_id")
-	private int batch_id;
-
-	public Pojo(String id, int counter, int batch_id) {
-		this.id = id;
-		this.counter = counter;
-		this.batch_id = batch_id;
-	}
-
-	public String getId() {
-		return id;
-	}
-
-	public void setId(String id) {
-		this.id = id;
-	}
-
-	public int getCounter() {
-		return counter;
-	}
-
-	public void setCounter(int counter) {
-		this.counter = counter;
-	}
-
-	public int getBatch_id() {
-		return batch_id;
-	}
-
-	public void setBatch_id(int batch_id) {
-		this.batch_id = batch_id;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java
deleted file mode 100644
index e1bcea9..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.cassandra.example;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.Cluster.Builder;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.cassandra.CassandraSink;
-import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
-
-import java.util.ArrayList;
-
-/**
- * This is an example showing the to use the Pojo Cassandra Sink in the Streaming API.
- * 
- * Pojo's have to be annotated with datastax annotations to work with this sink.
- *
- * The example assumes that a table exists in a local cassandra database, according to the following query:
- * CREATE TABLE IF NOT EXISTS test.message(body txt PRIMARY KEY)
- */
-public class CassandraPojoSinkExample {
-	private static final ArrayList<Message> messages = new ArrayList<>(20);
-
-	static {
-		for (long i = 0; i < 20; i++) {
-			messages.add(new Message("cassandra-" + i));
-		}
-	}
-
-	public static void main(String[] args) throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Message> source = env.fromCollection(messages);
-
-		CassandraSink.addSink(source)
-			.setClusterBuilder(new ClusterBuilder() {
-				@Override
-				protected Cluster buildCluster(Builder builder) {
-					return builder.addContactPoint("127.0.0.1").build();
-				}
-			})
-			.build();
-
-		env.execute("Cassandra Sink example");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java
deleted file mode 100644
index c6345df..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.cassandra.example;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.Cluster.Builder;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.cassandra.CassandraSink;
-import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
-
-import java.util.ArrayList;
-
-/**
- * This is an example showing the to use the Tuple Cassandra Sink in the Streaming API.
- *
- * The example assumes that a table exists in a local cassandra database, according to the following query:
- * CREATE TABLE IF NOT EXISTS test.writetuple(element1 text PRIMARY KEY, element2 int)
- */
-public class CassandraTupleSinkExample {
-	private static final String INSERT = "INSERT INTO test.writetuple (element1, element2) VALUES (?, ?)";
-	private static final ArrayList<Tuple2<String, Integer>> collection = new ArrayList<>(20);
-
-	static {
-		for (int i = 0; i < 20; i++) {
-			collection.add(new Tuple2<>("cassandra-" + i, i));
-		}
-	}
-
-	public static void main(String[] args) throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Tuple2<String, Integer>> source = env.fromCollection(collection);
-
-		CassandraSink.addSink(source)
-			.setQuery(INSERT)
-			.setClusterBuilder(new ClusterBuilder() {
-				@Override
-				protected Cluster buildCluster(Builder builder) {
-					return builder.addContactPoint("127.0.0.1").build();
-				}
-			})
-			.build();
-
-		env.execute("WriteTupleIntoCassandra");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
deleted file mode 100644
index 811c410..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.cassandra.example;
-
-import com.datastax.driver.core.Cluster;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.cassandra.CassandraSink;
-import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
-
-import java.util.UUID;
-
-/**
- * This is an example showing the to use the Cassandra Sink (with write-ahead log) in the Streaming API.
- *
- * The example assumes that a table exists in a local cassandra database, according to the following query:
- * CREATE TABLE example.values (id text, count int, PRIMARY KEY(id));
- * 
- * Important things to note are that checkpointing is enabled, a StateBackend is set and the enableWriteAheadLog() call
- * when creating the CassandraSink.
- */
-public class CassandraTupleWriteAheadSinkExample {
-	public static void main(String[] args) throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-		env.enableCheckpointing(1000);
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));
-		env.setStateBackend(new FsStateBackend("file:///" + System.getProperty("java.io.tmpdir") + "/flink/backend"));
-
-		CassandraSink<Tuple2<String, Integer>> sink = CassandraSink.addSink(env.addSource(new MySource()))
-			.setQuery("INSERT INTO example.values (id, counter) values (?, ?);")
-			.enableWriteAheadLog()
-			.setClusterBuilder(new ClusterBuilder() {
-				@Override
-				public Cluster buildCluster(Cluster.Builder builder) {
-					return builder.addContactPoint("127.0.0.1").build();
-				}
-			})
-			.build();
-
-		sink.name("Cassandra Sink").disableChaining().setParallelism(1).uid("hello");
-
-		env.execute();
-	}
-
-	public static class MySource implements SourceFunction<Tuple2<String, Integer>>, Checkpointed<Integer> {
-		private int counter = 0;
-		private boolean stop = false;
-
-		@Override
-		public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
-			while (!stop) {
-				Thread.sleep(50);
-				ctx.collect(new Tuple2<>("" + UUID.randomUUID(), 1));
-				counter++;
-				if (counter == 100) {
-					stop = true;
-				}
-			}
-		}
-
-		@Override
-		public void cancel() {
-			stop = true;
-		}
-
-		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return counter;
-		}
-
-		@Override
-		public void restoreState(Integer state) throws Exception {
-			this.counter = state;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/Message.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/Message.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/Message.java
deleted file mode 100644
index 7524d95..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/Message.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.cassandra.example;
-
-import com.datastax.driver.mapping.annotations.Column;
-import com.datastax.driver.mapping.annotations.Table;
-
-import java.io.Serializable;
-
-@Table(keyspace = "test", name = "message")
-public class Message implements Serializable {
-
-	private static final long serialVersionUID = 1123119384361005680L;
-
-	@Column(name = "body")
-	private String message;
-
-	public Message(String word) {
-		this.message = word;
-	}
-
-	public String getMessage() {
-		return message;
-	}
-
-	public void setMessage(String word) {
-		this.message = word;
-	}
-
-	public boolean equals(Object other) {
-		if (other instanceof Message) {
-			Message that = (Message) other;
-			return this.message.equals(that.message);
-		}
-		return false;
-	}
-
-	@Override
-	public int hashCode() {
-		return message.hashCode();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/test/resources/cassandra.yaml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/resources/cassandra.yaml b/flink-streaming-connectors/flink-connector-cassandra/src/test/resources/cassandra.yaml
deleted file mode 100644
index 0594ea3..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/resources/cassandra.yaml
+++ /dev/null
@@ -1,43 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-cluster_name: 'Test Cluster'
-commitlog_sync: 'periodic'
-commitlog_sync_period_in_ms: 10000
-commitlog_segment_size_in_mb: 16
-partitioner: 'org.apache.cassandra.dht.RandomPartitioner'
-endpoint_snitch: 'org.apache.cassandra.locator.SimpleSnitch'
-commitlog_directory: $PATH/commit'
-data_file_directories:
-    - $PATH/data'
-saved_caches_directory: $PATH/cache'
-listen_address: '127.0.0.1'
-seed_provider:
-    - class_name: 'org.apache.cassandra.locator.SimpleSeedProvider'
-      parameters:
-          - seeds: '127.0.0.1'
-native_transport_port: 9042
-
-concurrent_reads: 8
-concurrent_writes: 8
-
-auto_bootstrap: false
-auto_snapshot: false
-
-start_rpc: false
-start_native_transport: true
-native_transport_max_threads: 8

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

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

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

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

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

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

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

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


[12/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java
deleted file mode 100644
index 42b9682..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-
-import org.junit.Test;
-
-
-@SuppressWarnings("serial")
-public class Kafka010ProducerITCase extends KafkaProducerTestBase {
-
-	@Test
-	public void testCustomPartitioning() {
-		runCustomPartitioningTest();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
deleted file mode 100644
index f15fd45..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
+++ /dev/null
@@ -1,420 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import kafka.admin.AdminUtils;
-import kafka.common.KafkaException;
-import kafka.server.KafkaConfig;
-import kafka.server.KafkaServer;
-import kafka.utils.SystemTime$;
-import kafka.utils.ZkUtils;
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.commons.io.FileUtils;
-import org.apache.curator.test.TestingServer;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.util.NetUtils;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.protocol.SecurityProtocol;
-import org.apache.kafka.common.requests.MetadataResponse;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.net.BindException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.UUID;
-
-import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * An implementation of the KafkaServerProvider for Kafka 0.10
- */
-public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment {
-
-	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class);
-	private File tmpZkDir;
-	private File tmpKafkaParent;
-	private List<File> tmpKafkaDirs;
-	private List<KafkaServer> brokers;
-	private TestingServer zookeeper;
-	private String zookeeperConnectionString;
-	private String brokerConnectionString = "";
-	private Properties standardProps;
-	private Properties additionalServerProperties;
-	private boolean secureMode = false;
-	// 6 seconds is default. Seems to be too small for travis. 30 seconds
-	private int zkTimeout = 30000;
-
-	public String getBrokerConnectionString() {
-		return brokerConnectionString;
-	}
-
-	@Override
-	public Properties getStandardProperties() {
-		return standardProps;
-	}
-
-	@Override
-	public Properties getSecureProperties() {
-		Properties prop = new Properties();
-		if(secureMode) {
-			prop.put("security.inter.broker.protocol", "SASL_PLAINTEXT");
-			prop.put("security.protocol", "SASL_PLAINTEXT");
-			prop.put("sasl.kerberos.service.name", "kafka");
-
-			//add special timeout for Travis
-			prop.setProperty("zookeeper.session.timeout.ms", String.valueOf(zkTimeout));
-			prop.setProperty("zookeeper.connection.timeout.ms", String.valueOf(zkTimeout));
-			prop.setProperty("metadata.fetch.timeout.ms","120000");
-		}
-		return prop;
-	}
-
-	@Override
-	public String getVersion() {
-		return "0.10";
-	}
-
-	@Override
-	public List<KafkaServer> getBrokers() {
-		return brokers;
-	}
-
-	@Override
-	public <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, KeyedDeserializationSchema<T> readSchema, Properties props) {
-		return new FlinkKafkaConsumer010<>(topics, readSchema, props);
-	}
-
-	@Override
-	public <T> StreamSink<T> getProducerSink(String topic, KeyedSerializationSchema<T> serSchema, Properties props, KafkaPartitioner<T> partitioner) {
-		FlinkKafkaProducer010<T> prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner);
-		prod.setFlushOnCheckpoint(true);
-		return new StreamSink<>(prod);
-	}
-
-
-	@Override
-	public <T> DataStreamSink<T> produceIntoKafka(DataStream<T> stream, String topic, KeyedSerializationSchema<T> serSchema, Properties props, KafkaPartitioner<T> partitioner) {
-		FlinkKafkaProducer010<T> prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner);
-		prod.setFlushOnCheckpoint(true);
-		return stream.addSink(prod);
-	}
-
-	@Override
-	public KafkaOffsetHandler createOffsetHandler(Properties props) {
-		return new KafkaOffsetHandlerImpl(props);
-	}
-
-	@Override
-	public void restartBroker(int leaderId) throws Exception {
-		brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId)));
-	}
-
-	@Override
-	public int getLeaderToShutDown(String topic) throws Exception {
-		ZkUtils zkUtils = getZkUtils();
-		try {
-			MetadataResponse.PartitionMetadata firstPart = null;
-			do {
-				if (firstPart != null) {
-					LOG.info("Unable to find leader. error code {}", firstPart.error().code());
-					// not the first try. Sleep a bit
-					Thread.sleep(150);
-				}
-
-				List<MetadataResponse.PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkUtils).partitionMetadata();
-				firstPart = partitionMetadata.get(0);
-			}
-			while (firstPart.error().code() != 0);
-
-			return firstPart.leader().id();
-		} finally {
-			zkUtils.close();
-		}
-	}
-
-	@Override
-	public int getBrokerId(KafkaServer server) {
-		return server.config().brokerId();
-	}
-
-	@Override
-	public boolean isSecureRunSupported() {
-		return true;
-	}
-
-	@Override
-	public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) {
-		//increase the timeout since in Travis ZK connection takes long time for secure connection.
-		if(secureMode) {
-			//run only one kafka server to avoid multiple ZK connections from many instances - Travis timeout
-			numKafkaServers = 1;
-			zkTimeout = zkTimeout * 15;
-		}
-
-		this.additionalServerProperties = additionalServerProperties;
-		this.secureMode = secureMode;
-		File tempDir = new File(System.getProperty("java.io.tmpdir"));
-
-		tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
-		assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs());
-
-		tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString()));
-		assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs());
-
-		tmpKafkaDirs = new ArrayList<>(numKafkaServers);
-		for (int i = 0; i < numKafkaServers; i++) {
-			File tmpDir = new File(tmpKafkaParent, "server-" + i);
-			assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
-			tmpKafkaDirs.add(tmpDir);
-		}
-
-		zookeeper = null;
-		brokers = null;
-
-		try {
-			zookeeper = new TestingServer(-	1, tmpZkDir);
-			zookeeperConnectionString = zookeeper.getConnectString();
-			LOG.info("Starting Zookeeper with zookeeperConnectionString: {}", zookeeperConnectionString);
-
-			LOG.info("Starting KafkaServer");
-			brokers = new ArrayList<>(numKafkaServers);
-
-			for (int i = 0; i < numKafkaServers; i++) {
-				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i)));
-
-				if(secureMode) {
-					brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.SASL_PLAINTEXT)) + ",";
-				} else {
-					brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ",";
-				}
-			}
-
-			LOG.info("ZK and KafkaServer started.");
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("Test setup failed: " + t.getMessage());
-		}
-
-		standardProps = new Properties();
-		standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
-		standardProps.setProperty("bootstrap.servers", brokerConnectionString);
-		standardProps.setProperty("group.id", "flink-tests");
-		standardProps.setProperty("enable.auto.commit", "false");
-		standardProps.setProperty("zookeeper.session.timeout.ms", String.valueOf(zkTimeout));
-		standardProps.setProperty("zookeeper.connection.timeout.ms", String.valueOf(zkTimeout));
-		standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning. (earliest is kafka 0.10 value)
-		standardProps.setProperty("max.partition.fetch.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
-	}
-
-	@Override
-	public void shutdown() {
-		for (KafkaServer broker : brokers) {
-			if (broker != null) {
-				broker.shutdown();
-			}
-		}
-		brokers.clear();
-
-		if (zookeeper != null) {
-			try {
-				zookeeper.stop();
-			}
-			catch (Exception e) {
-				LOG.warn("ZK.stop() failed", e);
-			}
-			zookeeper = null;
-		}
-
-		// clean up the temp spaces
-
-		if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
-			try {
-				FileUtils.deleteDirectory(tmpKafkaParent);
-			}
-			catch (Exception e) {
-				// ignore
-			}
-		}
-		if (tmpZkDir != null && tmpZkDir.exists()) {
-			try {
-				FileUtils.deleteDirectory(tmpZkDir);
-			}
-			catch (Exception e) {
-				// ignore
-			}
-		}
-	}
-
-	public ZkUtils getZkUtils() {
-		ZkClient creator = new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
-				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
-		return ZkUtils.apply(creator, false);
-	}
-
-	@Override
-	public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor, Properties topicConfig) {
-		// create topic with one client
-		LOG.info("Creating topic {}", topic);
-
-		ZkUtils zkUtils = getZkUtils();
-		try {
-			AdminUtils.createTopic(zkUtils, topic, numberOfPartitions, replicationFactor, topicConfig, kafka.admin.RackAwareMode.Enforced$.MODULE$);
-		} finally {
-			zkUtils.close();
-		}
-
-		// validate that the topic has been created
-		final long deadline = System.currentTimeMillis() + 30000;
-		do {
-			try {
-				if(secureMode) {
-					//increase wait time since in Travis ZK timeout occurs frequently
-					int wait = zkTimeout / 100;
-					LOG.info("waiting for {} msecs before the topic {} can be checked", wait, topic);
-					Thread.sleep(wait);
-				} else {
-					Thread.sleep(100);
-				}
-			} catch (InterruptedException e) {
-				// restore interrupted state
-			}
-			// we could use AdminUtils.topicExists(zkUtils, topic) here, but it's results are
-			// not always correct.
-
-			// create a new ZK utils connection
-			ZkUtils checkZKConn = getZkUtils();
-			if(AdminUtils.topicExists(checkZKConn, topic)) {
-				checkZKConn.close();
-				return;
-			}
-			checkZKConn.close();
-		}
-		while (System.currentTimeMillis() < deadline);
-		fail("Test topic could not be created");
-	}
-
-	@Override
-	public void deleteTestTopic(String topic) {
-		ZkUtils zkUtils = getZkUtils();
-		try {
-			LOG.info("Deleting topic {}", topic);
-
-			ZkClient zk = new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
-				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
-
-			AdminUtils.deleteTopic(zkUtils, topic);
-
-			zk.close();
-		} finally {
-			zkUtils.close();
-		}
-	}
-
-	/**
-	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
-	 */
-	protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception {
-		Properties kafkaProperties = new Properties();
-
-		// properties have to be Strings
-		kafkaProperties.put("advertised.host.name", KAFKA_HOST);
-		kafkaProperties.put("broker.id", Integer.toString(brokerId));
-		kafkaProperties.put("log.dir", tmpFolder.toString());
-		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
-		kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
-		kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
-
-		// for CI stability, increase zookeeper session timeout
-		kafkaProperties.put("zookeeper.session.timeout.ms", zkTimeout);
-		kafkaProperties.put("zookeeper.connection.timeout.ms", zkTimeout);
-		if(additionalServerProperties != null) {
-			kafkaProperties.putAll(additionalServerProperties);
-		}
-
-		final int numTries = 5;
-
-		for (int i = 1; i <= numTries; i++) {
-			int kafkaPort = NetUtils.getAvailablePort();
-			kafkaProperties.put("port", Integer.toString(kafkaPort));
-
-			//to support secure kafka cluster
-			if(secureMode) {
-				LOG.info("Adding Kafka secure configurations");
-				kafkaProperties.put("listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort);
-				kafkaProperties.put("advertised.listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort);
-				kafkaProperties.putAll(getSecureProperties());
-			}
-
-			KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
-
-			try {
-				scala.Option<String> stringNone = scala.Option.apply(null);
-				KafkaServer server = new KafkaServer(kafkaConfig, SystemTime$.MODULE$, stringNone);
-				server.startup();
-				return server;
-			}
-			catch (KafkaException e) {
-				if (e.getCause() instanceof BindException) {
-					// port conflict, retry...
-					LOG.info("Port conflict when starting Kafka Broker. Retrying...");
-				}
-				else {
-					throw e;
-				}
-			}
-		}
-
-		throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
-	}
-
-	private class KafkaOffsetHandlerImpl implements KafkaOffsetHandler {
-
-		private final KafkaConsumer<byte[], byte[]> offsetClient;
-
-		public KafkaOffsetHandlerImpl(Properties props) {
-			offsetClient = new KafkaConsumer<>(props);
-		}
-
-		@Override
-		public Long getCommittedOffset(String topicName, int partition) {
-			OffsetAndMetadata committed = offsetClient.committed(new TopicPartition(topicName, partition));
-			return (committed != null) ? committed.offset() : null;
-		}
-
-		@Override
-		public void close() {
-			offsetClient.close();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/log4j-test.properties
deleted file mode 100644
index fbeb110..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,30 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-log4j.rootLogger=INFO, testlogger
-
-log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
-log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
-log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
-
-# suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
-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/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/logback-test.xml
deleted file mode 100644
index 45b3b92..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,30 +0,0 @@
-<!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one
-  ~ or more contributor license agreements.  See the NOTICE file
-  ~ distributed with this work for additional information
-  ~ regarding copyright ownership.  The ASF licenses this file
-  ~ to you under the Apache License, Version 2.0 (the
-  ~ "License"); you may not use this file except in compliance
-  ~ with the License.  You may obtain a copy of the License at
-  ~
-  ~     http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
-
-<configuration>
-    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <root level="WARN">
-        <appender-ref ref="STDOUT"/>
-    </root>
-    <logger name="org.apache.flink.streaming" level="WARN"/>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml
deleted file mode 100644
index f17f9ae..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml
+++ /dev/null
@@ -1,219 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-kafka-0.8_2.10</artifactId>
-	<name>flink-connector-kafka-0.8</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<kafka.version>0.8.2.2</kafka.version>
-	</properties>
-
-	<dependencies>
-
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-curator-recipes</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-			<!-- Projects depending on this project,
-			won't depend on flink-table. -->
-			<optional>true</optional>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.kafka</groupId>
-			<artifactId>kafka_${scala.binary.version}</artifactId>
-			<version>${kafka.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>com.sun.jmx</groupId>
-					<artifactId>jmxri</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jdmk</groupId>
-					<artifactId>jmxtools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-simple</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>net.sf.jopt-simple</groupId>
-					<artifactId>jopt-simple</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.scala-lang</groupId>
-					<artifactId>scala-reflect</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.scala-lang</groupId>
-					<artifactId>scala-compiler</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.yammer.metrics</groupId>
-					<artifactId>metrics-annotation</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.xerial.snappy</groupId>
-					<artifactId>snappy-java</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<!-- test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.curator</groupId>
-			<artifactId>curator-test</artifactId>
-			<version>${curator.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-metrics-jmx</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-	</dependencies>
-
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
-					<forkCount>1</forkCount>
-				</configuration>
-			</plugin>
-			<!-- Relocate curator -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>shade-flink</id>
-						<phase>package</phase>
-						<goals>
-							<goal>shade</goal>
-						</goals>
-						<configuration>
-							<artifactSet>
-								<includes combine.children="append">
-									<include>org.apache.flink:flink-shaded-curator-recipes</include>
-								</includes>
-							</artifactSet>
-							<relocations combine.children="append">
-								<relocation>
-									<pattern>org.apache.curator</pattern>
-									<shadedPattern>org.apache.flink.shaded.org.apache.curator</shadedPattern>
-								</relocation>
-							</relocations>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-	
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index 0aacccd..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
+++ /dev/null
@@ -1,398 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import kafka.api.OffsetRequest;
-import kafka.cluster.Broker;
-import kafka.common.ErrorMapping;
-import kafka.javaapi.PartitionMetadata;
-import kafka.javaapi.TopicMetadata;
-import kafka.javaapi.TopicMetadataRequest;
-import kafka.javaapi.consumer.SimpleConsumer;
-
-import org.apache.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.Kafka08Fetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader;
-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.PropertiesUtil;
-import org.apache.flink.util.SerializedValue;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.common.Node;
-
-import java.net.InetAddress;
-import java.net.URL;
-import java.net.UnknownHostException;
-import java.nio.channels.ClosedChannelException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-import static org.apache.flink.util.PropertiesUtil.getInt;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
- * Apache Kafka 0.8.x. The consumer can run in multiple parallel instances, each of which will pull
- * data from one or more Kafka partitions. 
- * 
- * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
- * during a failure, and that the computation processes elements "exactly once". 
- * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
- * 
- * <p>Flink's Kafka Consumer is designed to be compatible with Kafka's High-Level Consumer API (0.8.x).
- * Most of Kafka's configuration variables can be used with this consumer as well:
- *         <ul>
- *             <li>socket.timeout.ms</li>
- *             <li>socket.receive.buffer.bytes</li>
- *             <li>fetch.message.max.bytes</li>
- *             <li>auto.offset.reset with the values "largest", "smallest"</li>
- *             <li>fetch.wait.max.ms</li>
- *         </ul>
- *     </li>
- * </ul>
- * 
- * <h1>Offset handling</h1>
- * 
- * <p>Offsets whose records have been read and are checkpointed will be committed back to ZooKeeper
- * by the offset handler. In addition, the offset handler finds the point where the source initially
- * starts reading from the stream, when the streaming job is started.</p>
- *
- * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
- * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
- * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
- * has consumed a topic.</p>
- *
- * <p>If checkpointing is disabled, the consumer will periodically commit the current offset
- * to Zookeeper.</p>
- *
- * <p>When using a Kafka topic to send data between Flink jobs, we recommend using the
- * {@see TypeInformationSerializationSchema} and {@see TypeInformationKeyValueSerializationSchema}.</p>
- * 
- * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
- * is constructed. That means that the client that submits the program needs to be able to
- * reach the Kafka brokers or ZooKeeper.</p>
- */
-public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
-
-	private static final long serialVersionUID = -6272159445203409112L;
-
-	/** Configuration key for the number of retries for getting the partition info */
-	public static final String GET_PARTITIONS_RETRIES_KEY = "flink.get-partitions.retry";
-
-	/** Default number of retries for getting the partition info. One retry means going through the full list of brokers */
-	public static final int DEFAULT_GET_PARTITIONS_RETRIES = 3;
-
-	// ------------------------------------------------------------------------
-
-	/** The properties to parametrize the Kafka consumer and ZooKeeper client */ 
-	private final Properties kafkaProperties;
-
-	/** The behavior when encountering an invalid offset (see {@link OffsetRequest}) */
-	private final long invalidOffsetBehavior;
-
-	/** The interval in which to automatically commit (-1 if deactivated) */
-	private final long autoCommitInterval;
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.8.x
-	 *
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param valueDeserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer08(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		this(Collections.singletonList(topic), valueDeserializer, props);
-	}
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.8.x
-	 *
-	 * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value
-	 * pairs, offsets, and topic names from Kafka.
-	 *
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param deserializer
-	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer08(String topic, KeyedDeserializationSchema<T> deserializer, Properties props) {
-		this(Collections.singletonList(topic), deserializer, props);
-	}
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.8.x
-	 *
-	 * This constructor allows passing multiple topics to the consumer.
-	 *
-	 * @param topics
-	 *           The Kafka topics to read from.
-	 * @param deserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties that are used to configure both the fetcher and the offset handler.
-	 */
-	public FlinkKafkaConsumer08(List<String> topics, DeserializationSchema<T> deserializer, Properties props) {
-		this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props);
-	}
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.8.x
-	 *
-	 * This constructor allows passing multiple topics and a key/value deserialization schema.
-	 * 
-	 * @param topics
-	 *           The Kafka topics to read from.
-	 * @param deserializer
-	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           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(topics, deserializer);
-
-		checkNotNull(topics, "topics");
-		this.kafkaProperties = checkNotNull(props, "props");
-
-		// validate the zookeeper properties
-		validateZooKeeperConfig(props);
-
-		this.invalidOffsetBehavior = getInvalidOffsetBehavior(props);
-		this.autoCommitInterval = PropertiesUtil.getLong(props, "auto.commit.interval.ms", 60000);
-	}
-
-	@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(kafkaProperties.getProperty(KEY_DISABLE_METRICS, "false"));
-
-		return new Kafka08Fetcher<>(sourceContext, thisSubtaskPartitions,
-				watermarksPeriodic, watermarksPunctuated,
-				runtimeContext, deserializer, kafkaProperties,
-				invalidOffsetBehavior, autoCommitInterval, useMetrics);
-	}
-
-	@Override
-	protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
-		// Connect to a broker to get the partitions for all topics
-		List<KafkaTopicPartition> partitionInfos =
-			KafkaTopicPartition.dropLeaderData(getPartitionsForTopic(topics, kafkaProperties));
-
-		if (partitionInfos.size() == 0) {
-			throw new RuntimeException(
-				"Unable to retrieve any partitions for the requested topics " + topics +
-					". Please check previous log entries");
-		}
-
-		if (LOG.isInfoEnabled()) {
-			logPartitionInfo(LOG, partitionInfos);
-		}
-
-		return partitionInfos;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Kafka / ZooKeeper communication utilities
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Send request to Kafka to get partitions for topic.
-	 * 
-	 * @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(List<String> topics, Properties properties) {
-		String seedBrokersConfString = properties.getProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
-		final int numRetries = getInt(properties, GET_PARTITIONS_RETRIES_KEY, DEFAULT_GET_PARTITIONS_RETRIES);
-		
-		checkNotNull(seedBrokersConfString, "Configuration property %s not set", ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
-		String[] seedBrokers = seedBrokersConfString.split(",");
-		List<KafkaTopicPartitionLeader> partitions = new ArrayList<>();
-
-		final String clientId = "flink-kafka-consumer-partition-lookup";
-		final int soTimeout = getInt(properties, "socket.timeout.ms", 30000);
-		final int bufferSize = getInt(properties, "socket.receive.buffer.bytes", 65536);
-
-		Random rnd = new Random();
-		retryLoop: for (int retry = 0; retry < numRetries; retry++) {
-			// we pick a seed broker randomly to avoid overloading the first broker with all the requests when the
-			// parallel source instances start. Still, we try all available brokers.
-			int index = rnd.nextInt(seedBrokers.length);
-			brokersLoop: for (int arrIdx = 0; arrIdx < seedBrokers.length; arrIdx++) {
-				String seedBroker = seedBrokers[index];
-				LOG.info("Trying to get topic metadata from broker {} in try {}/{}", seedBroker, retry, numRetries);
-				if (++index == seedBrokers.length) {
-					index = 0;
-				}
-
-				URL brokerUrl = NetUtils.getCorrectHostnamePort(seedBroker);
-				SimpleConsumer consumer = null;
-				try {
-					consumer = new SimpleConsumer(brokerUrl.getHost(), brokerUrl.getPort(), soTimeout, bufferSize, clientId);
-
-					TopicMetadataRequest req = new TopicMetadataRequest(topics);
-					kafka.javaapi.TopicMetadataResponse resp = consumer.send(req);
-
-					List<TopicMetadata> metaData = resp.topicsMetadata();
-
-					// clear in case we have an incomplete list from previous tries
-					partitions.clear();
-					for (TopicMetadata item : metaData) {
-						if (item.errorCode() != ErrorMapping.NoError()) {
-							// warn and try more brokers
-							LOG.warn("Error while getting metadata from broker " + seedBroker + " to find partitions " +
-									"for " + topics.toString() + ". Error: " + ErrorMapping.exceptionFor(item.errorCode()).getMessage());
-							continue brokersLoop;
-						}
-						if (!topics.contains(item.topic())) {
-							LOG.warn("Received metadata from topic " + item.topic() + " even though it was not requested. Skipping ...");
-							continue brokersLoop;
-						}
-						for (PartitionMetadata part : item.partitionsMetadata()) {
-							Node leader = brokerToNode(part.leader());
-							KafkaTopicPartition ktp = new KafkaTopicPartition(item.topic(), part.partitionId());
-							KafkaTopicPartitionLeader pInfo = new KafkaTopicPartitionLeader(ktp, leader);
-							partitions.add(pInfo);
-						}
-					}
-					break retryLoop; // leave the loop through the brokers
-				} catch (Exception e) {
-					//validates seed brokers in case of a ClosedChannelException
-					validateSeedBrokers(seedBrokers, e);
-					LOG.warn("Error communicating with broker " + seedBroker + " to find partitions for " + topics.toString() + "." +
-							"" + e.getClass() + ". Message: " + e.getMessage());
-					LOG.debug("Detailed trace", e);
-					// we sleep a bit. Retrying immediately doesn't make sense in cases where Kafka is reorganizing the leader metadata
-					try {
-						Thread.sleep(500);
-					} catch (InterruptedException e1) {
-						// sleep shorter.
-					}
-				} finally {
-					if (consumer != null) {
-						consumer.close();
-					}
-				}
-			} // brokers loop
-		} // retries loop
-		return partitions;
-	}
-
-	/**
-	 * Turn a broker instance into a node instance
-	 * @param broker broker instance
-	 * @return Node representing the given broker
-	 */
-	private static Node brokerToNode(Broker broker) {
-		return new Node(broker.id(), broker.host(), broker.port());
-	}
-
-	/**
-	 * Validate the ZK configuration, checking for required parameters
-	 * @param props Properties to check
-	 */
-	protected static void validateZooKeeperConfig(Properties props) {
-		if (props.getProperty("zookeeper.connect") == null) {
-			throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set in the properties");
-		}
-		if (props.getProperty(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-			throw new IllegalArgumentException("Required property '" + ConsumerConfig.GROUP_ID_CONFIG
-					+ "' has not been set in the properties");
-		}
-		
-		try {
-			//noinspection ResultOfMethodCallIgnored
-			Integer.parseInt(props.getProperty("zookeeper.session.timeout.ms", "0"));
-		}
-		catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Property 'zookeeper.session.timeout.ms' is not a valid integer");
-		}
-		
-		try {
-			//noinspection ResultOfMethodCallIgnored
-			Integer.parseInt(props.getProperty("zookeeper.connection.timeout.ms", "0"));
-		}
-		catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Property 'zookeeper.connection.timeout.ms' is not a valid integer");
-		}
-	}
-
-	/**
-	 * Validate that at least one seed broker is valid in case of a
-	 * ClosedChannelException.
-	 * 
-	 * @param seedBrokers
-	 *            array containing the seed brokers e.g. ["host1:port1",
-	 *            "host2:port2"]
-	 * @param exception
-	 *            instance
-	 */
-	private static void validateSeedBrokers(String[] seedBrokers, Exception exception) {
-		if (!(exception instanceof ClosedChannelException)) {
-			return;
-		}
-		int unknownHosts = 0;
-		for (String broker : seedBrokers) {
-			URL brokerUrl = NetUtils.getCorrectHostnamePort(broker.trim());
-			try {
-				InetAddress.getByName(brokerUrl.getHost());
-			} catch (UnknownHostException e) {
-				unknownHosts++;
-			}
-		}
-		// throw meaningful exception if all the provided hosts are invalid
-		if (unknownHosts == seedBrokers.length) {
-			throw new IllegalArgumentException("All the servers provided in: '"
-					+ ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + "' config are invalid. (unknown hosts)");
-		}
-	}
-
-	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/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
deleted file mode 100644
index 56ccd0b..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * THIS CLASS IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
- */
-@Deprecated
-public class FlinkKafkaConsumer081<T> extends FlinkKafkaConsumer08<T> {
-
-	private static final long serialVersionUID = -5649906773771949146L;
-
-	/**
-	 * THIS CONSTRUCTOR IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
-	 */
-	@Deprecated
-	public FlinkKafkaConsumer081(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		super(topic, valueDeserializer, props);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
deleted file mode 100644
index 0520336..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * THIS CLASS IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
- */
-@Deprecated
-public class FlinkKafkaConsumer082<T> extends FlinkKafkaConsumer08<T> {
-
-	private static final long serialVersionUID = -5649906773771949146L;
-
-	/**
-	 * THIS CONSTRUCTOR IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
-	 */
-	@Deprecated
-	public FlinkKafkaConsumer082(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		super(topic, valueDeserializer, props);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
deleted file mode 100644
index 1c2e0b7..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import java.util.Properties;
-
-
-/**
- * THIS CLASS IS DEPRECATED. Use FlinkKafkaProducer08 instead.
- */
-@Deprecated
-public class FlinkKafkaProducer<IN> extends FlinkKafkaProducer08<IN>  {
-
-	@Deprecated
-	public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
-		super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), null);
-	}
-
-	@Deprecated
-	public FlinkKafkaProducer(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
-		super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, null);
-	}
-
-	@Deprecated
-	public FlinkKafkaProducer(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
-		super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
-
-	}
-
-	@Deprecated
-	public FlinkKafkaProducer(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
-		super(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), null);
-	}
-
-	@Deprecated
-	public FlinkKafkaProducer(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
-		super(topicId, serializationSchema, producerConfig, null);
-	}
-
-	@Deprecated
-	public FlinkKafkaProducer(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
-		super(topicId, serializationSchema, producerConfig, customPartitioner);
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
deleted file mode 100644
index 65de5fc..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-import java.util.Properties;
-
-
-/**
- * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.8.
- *
- * Please note that this producer does not have any reliability guarantees.
- *
- * @param <IN> Type of the messages to write into Kafka.
- */
-public class FlinkKafkaProducer08<IN> extends FlinkKafkaProducerBase<IN>  {
-
-	private static final long serialVersionUID = 1L;
-
-	// ------------------- Keyless serialization schema constructors ----------------------
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 *			Comma separated addresses of the brokers
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined (keyless) serialization schema.
-	 */
-	public FlinkKafkaProducer08(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
-		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner<IN>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined (keyless) serialization schema.
-	 * @param producerConfig
-	 * 			Properties with the producer configuration.
-	 */
-	public FlinkKafkaProducer08(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
-		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner<IN>());
-	}
-
-	/**
-	 * The main constructor for creating a FlinkKafkaProducer.
-	 *
-	 * @param topicId The topic to write data to
-	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
-	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
-	 * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
-	 */
-	public FlinkKafkaProducer08(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
-		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
-
-	}
-
-	// ------------------- Key/Value serialization schema constructors ----------------------
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 *			Comma separated addresses of the brokers
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined serialization schema supporting key/value messages
-	 */
-	public FlinkKafkaProducer08(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
-		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner<IN>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined serialization schema supporting key/value messages
-	 * @param producerConfig
-	 * 			Properties with the producer configuration.
-	 */
-	public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
-		this(topicId, serializationSchema, producerConfig, new FixedPartitioner<IN>());
-	}
-
-	/**
-	 * The main constructor for creating a FlinkKafkaProducer.
-	 *
-	 * @param topicId The topic to write data to
-	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
-	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
-	 * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
-	 */
-	public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
-		super(topicId, serializationSchema, producerConfig, customPartitioner);
-	}
-
-	@Override
-	protected void flush() {
-		// The Kafka 0.8 producer doesn't support flushing, we wait here
-		// until all pending records are confirmed
-		synchronized (pendingRecordsLock) {
-			while (pendingRecords > 0) {
-				try {
-					pendingRecordsLock.wait();
-				} catch (InterruptedException e) {
-					// this can be interrupted when the Task has been cancelled.
-					// by throwing an exception, we ensure that this checkpoint doesn't get confirmed
-					throw new RuntimeException("Flushing got interrupted while checkpointing", e);
-				}
-			}
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
deleted file mode 100644
index b155576..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-import java.util.Properties;
-
-/**
- * Kafka 0.8 {@link KafkaTableSink} that serializes data in JSON format.
- */
-public class Kafka08JsonTableSink extends KafkaJsonTableSink {
-
-	/**
-	 * Creates {@link KafkaTableSink} for Kafka 0.8
-	 *
-	 * @param topic topic in Kafka to which table is written
-	 * @param properties properties to connect to Kafka
-	 * @param partitioner Kafka partitioner
-	 */
-	public Kafka08JsonTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner) {
-		super(topic, properties, partitioner);
-	}
-
-	@Override
-	protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties, SerializationSchema<Row> serializationSchema, KafkaPartitioner<Row> partitioner) {
-		return new FlinkKafkaProducer08<>(topic, serializationSchema, properties, partitioner);
-	}
-
-	@Override
-	protected Kafka08JsonTableSink createCopy() {
-		return new Kafka08JsonTableSink(topic, properties, partitioner);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
deleted file mode 100644
index 63bb57e..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Kafka {@link StreamTableSource} for Kafka 0.8.
- */
-public class Kafka08JsonTableSource extends KafkaJsonTableSource {
-
-	/**
-	 * Creates a Kafka 0.8 JSON {@link StreamTableSource}.
-	 *
-	 * @param topic      Kafka topic to consume.
-	 * @param properties Properties for the Kafka consumer.
-	 * @param fieldNames Row field names.
-	 * @param fieldTypes Row field types.
-	 */
-	public Kafka08JsonTableSource(
-			String topic,
-			Properties properties,
-			String[] fieldNames,
-			TypeInformation<?>[] fieldTypes) {
-
-		super(topic, properties, fieldNames, fieldTypes);
-	}
-
-	/**
-	 * Creates a Kafka 0.8 JSON {@link StreamTableSource}.
-	 *
-	 * @param topic      Kafka topic to consume.
-	 * @param properties Properties for the Kafka consumer.
-	 * @param fieldNames Row field names.
-	 * @param fieldTypes Row field types.
-	 */
-	public Kafka08JsonTableSource(
-			String topic,
-			Properties properties,
-			String[] fieldNames,
-			Class<?>[] fieldTypes) {
-
-		super(topic, properties, fieldNames, fieldTypes);
-	}
-
-	@Override
-	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
-		return new FlinkKafkaConsumer08<>(topic, deserializationSchema, properties);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
deleted file mode 100644
index 8f51237..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Kafka {@link StreamTableSource} for Kafka 0.8.
- */
-public class Kafka08TableSource extends KafkaTableSource {
-
-	/**
-	 * Creates a Kafka 0.8 {@link StreamTableSource}.
-	 *
-	 * @param topic                 Kafka topic to consume.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param deserializationSchema Deserialization schema to use for Kafka records.
-	 * @param fieldNames            Row field names.
-	 * @param fieldTypes            Row field types.
-	 */
-	public Kafka08TableSource(
-			String topic,
-			Properties properties,
-			DeserializationSchema<Row> deserializationSchema,
-			String[] fieldNames,
-			TypeInformation<?>[] fieldTypes) {
-
-		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
-	}
-
-	/**
-	 * Creates a Kafka 0.8 {@link StreamTableSource}.
-	 *
-	 * @param topic                 Kafka topic to consume.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param deserializationSchema Deserialization schema to use for Kafka records.
-	 * @param fieldNames            Row field names.
-	 * @param fieldTypes            Row field types.
-	 */
-	public Kafka08TableSource(
-			String topic,
-			Properties properties,
-			DeserializationSchema<Row> deserializationSchema,
-			String[] fieldNames,
-			Class<?>[] fieldTypes) {
-
-		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
-	}
-
-	@Override
-	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
-		return new FlinkKafkaConsumer08<>(topic, deserializationSchema, properties);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java
deleted file mode 100644
index 23ff276..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java
+++ /dev/null
@@ -1,507 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * A special form of blocking queue with two additions:
- * <ol>
- *     <li>The queue can be closed atomically when empty. Adding elements after the queue
- *         is closed fails. This allows queue consumers to atomically discover that no elements
- *         are available and mark themselves as shut down.</li>
- *     <li>The queue allows to poll batches of elements in one polling call.</li>
- * </ol>
- * 
- * The queue has no capacity restriction and is safe for multiple producers and consumers.
- * 
- * <p>Note: Null elements are prohibited.
- * 
- * @param <E> The type of elements in the queue.
- */
-public class ClosableBlockingQueue<E> {
-
-	/** The lock used to make queue accesses and open checks atomic */
-	private final ReentrantLock lock;
-	
-	/** The condition on which blocking get-calls wait if the queue is empty */
-	private final Condition nonEmpty;
-	
-	/** The deque of elements */
-	private final ArrayDeque<E> elements;
-	
-	/** Flag marking the status of the queue */
-	private volatile boolean open;
-	
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new empty queue.
-	 */
-	public ClosableBlockingQueue() {
-		this(10);
-	}
-
-	/**
-	 * Creates a new empty queue, reserving space for at least the specified number
-	 * of elements. The queu can still grow, of more elements are added than the
-	 * reserved space.
-	 * 
-	 * @param initialSize The number of elements to reserve space for.
-	 */
-	public ClosableBlockingQueue(int initialSize) {
-		this.lock = new ReentrantLock(true);
-		this.nonEmpty = this.lock.newCondition();
-		
-		this.elements = new ArrayDeque<>(initialSize);
-		this.open = true;
-		
-		
-	}
-
-	/**
-	 * Creates a new queue that contains the given elements.
-	 * 
-	 * @param initialElements The elements to initially add to the queue.
-	 */
-	public ClosableBlockingQueue(Collection<? extends E> initialElements) {
-		this(initialElements.size());
-		this.elements.addAll(initialElements);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Size and status
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the number of elements currently in the queue.
-	 * @return The number of elements currently in the queue.
-	 */
-	public int size() {
-		lock.lock();
-		try {
-			return elements.size();
-		} finally {
-			lock.unlock();
-		}
-	}
-
-	/**
-	 * Checks whether the queue is empty (has no elements).
-	 * @return True, if the queue is empty; false, if it is non-empty.
-	 */
-	public boolean isEmpty() {
-		return size() == 0;
-	}
-
-	/**
-	 * Checks whether the queue is currently open, meaning elements can be added and polled.
-	 * @return True, if the queue is open; false, if it is closed.
-	 */
-	public boolean isOpen() {
-		return open;
-	}
-	
-	/**
-	 * Tries to close the queue. Closing the queue only succeeds when no elements are
-	 * in the queue when this method is called. Checking whether the queue is empty, and
-	 * marking the queue as closed is one atomic operation.
-	 *
-	 * @return True, if the queue is closed, false if the queue remains open.
-	 */
-	public boolean close() {
-		lock.lock();
-		try {
-			if (open) {
-				if (elements.isEmpty()) {
-					open = false;
-					nonEmpty.signalAll();
-					return true;
-				} else {
-					return false;
-				}
-			}
-			else {
-				// already closed
-				return true;
-			}
-		} finally {
-			lock.unlock();
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Adding / Removing elements
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Tries to add an element to the queue, if the queue is still open. Checking whether the queue
-	 * is open and adding the element is one atomic operation.
-	 * 
-	 * <p>Unlike the {@link #add(Object)} method, this method never throws an exception,
-	 * but only indicates via the return code if the element was added or the
-	 * queue was closed.
-	 * 
-	 * @param element The element to add.
-	 * @return True, if the element was added, false if the queue was closes.
-	 */
-	public boolean addIfOpen(E element) {
-		requireNonNull(element);
-		
-		lock.lock();
-		try {
-			if (open) {
-				elements.addLast(element);
-				if (elements.size() == 1) {
-					nonEmpty.signalAll();
-				}
-			}
-			return open;
-		} finally {
-			lock.unlock();
-		}
-	}
-
-	/**
-	 * Adds the element to the queue, or fails with an exception, if the queue is closed.
-	 * Checking whether the queue is open and adding the element is one atomic operation.
-	 * 
-	 * @param element The element to add.
-	 * @throws IllegalStateException Thrown, if the queue is closed.
-	 */
-	public void add(E element) throws IllegalStateException {
-		requireNonNull(element);
-
-		lock.lock();
-		try {
-			if (open) {
-				elements.addLast(element);
-				if (elements.size() == 1) {
-					nonEmpty.signalAll();
-				}
-			} else {
-				throw new IllegalStateException("queue is closed");
-			}
-		} finally {
-			lock.unlock();
-		}
-	}
-
-	/**
-	 * Returns the queue's next element without removing it, if the queue is non-empty.
-	 * Otherwise, returns null. 
-	 *
-	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
-	 * Checking whether the queue is open and getting the next element is one atomic operation.
-	 * 
-	 * <p>This method never blocks.
-	 * 
-	 * @return The queue's next element, or null, if the queue is empty.
-	 * @throws IllegalStateException Thrown, if the queue is closed.
-	 */
-	public E peek() {
-		lock.lock();
-		try {
-			if (open) {
-				if (elements.size() > 0) {
-					return elements.getFirst();
-				} else {
-					return null;
-				}
-			} else {
-				throw new IllegalStateException("queue is closed");
-			}
-		} finally {
-			lock.unlock();
-		}
-	}
-
-	/**
-	 * Returns the queue's next element and removes it, the queue is non-empty.
-	 * Otherwise, this method returns null. 
-	 *
-	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
-	 * Checking whether the queue is open and removing the next element is one atomic operation.
-	 *
-	 * <p>This method never blocks.
-	 *
-	 * @return The queue's next element, or null, if the queue is empty.
-	 * @throws IllegalStateException Thrown, if the queue is closed.
-	 */
-	public E poll() {
-		lock.lock();
-		try {
-			if (open) {
-				if (elements.size() > 0) {
-					return elements.removeFirst();
-				} else {
-					return null;
-				}
-			} else {
-				throw new IllegalStateException("queue is closed");
-			}
-		} finally {
-			lock.unlock();
-		}
-	}
-
-	/**
-	 * Returns all of the queue's current elements in a list, if the queue is non-empty.
-	 * Otherwise, this method returns null. 
-	 *
-	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
-	 * Checking whether the queue is open and removing the elements is one atomic operation.
-	 *
-	 * <p>This method never blocks.
-	 *
-	 * @return All of the queue's elements, or null, if the queue is empty.
-	 * @throws IllegalStateException Thrown, if the queue is closed.
-	 */
-	public List<E> pollBatch() {
-		lock.lock();
-		try {
-			if (open) {
-				if (elements.size() > 0) {
-					ArrayList<E> result = new ArrayList<>(elements);
-					elements.clear();
-					return result;
-				} else {
-					return null;
-				}
-			} else {
-				throw new IllegalStateException("queue is closed");
-			}
-		} finally {
-			lock.unlock();
-		}
-	}
-
-	/**
-	 * Returns the next element in the queue. If the queue is empty, this method
-	 * waits until at least one element is added.
-	 * 
-	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
-	 * Checking whether the queue is open and removing the next element is one atomic operation.
-	 * 
-	 * @return The next element in the queue, never null.
-	 * 
-	 * @throws IllegalStateException Thrown, if the queue is closed.
-	 * @throws InterruptedException Throw, if the thread is interrupted while waiting for an
-	 *                              element to be added.
-	 */
-	public E getElementBlocking() throws InterruptedException {
-		lock.lock();
-		try {
-			while (open && elements.isEmpty()) {
-				nonEmpty.await();
-			}
-			
-			if (open) {
-				return elements.removeFirst();
-			} else {
-				throw new IllegalStateException("queue is closed");
-			}
-		} finally {
-			lock.unlock();
-		}
-	}
-
-	/**
-	 * Returns the next element in the queue. If the queue is empty, this method
-	 * waits at most a certain time until an element becomes available. If no element
-	 * is available after that time, the method returns null.
-	 * 
-	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
-	 * Checking whether the queue is open and removing the next element is one atomic operation.
-	 * 
-	 * @param timeoutMillis The number of milliseconds to block, at most.
-	 * @return The next element in the queue, or null, if the timeout expires  before an element is available.
-	 * 
-	 * @throws IllegalStateException Thrown, if the queue is closed.
-	 * @throws InterruptedException Throw, if the thread is interrupted while waiting for an
-	 *                              element to be added.
-	 */
-	public E getElementBlocking(long timeoutMillis) throws InterruptedException {
-		if (timeoutMillis == 0L) {
-			// wait forever case
-			return getElementBlocking();
-		} else if (timeoutMillis < 0L) {
-			throw new IllegalArgumentException("invalid timeout");
-		}
-		
-		final long deadline = System.currentTimeMillis() + timeoutMillis;
-		
-		lock.lock();
-		try {
-			while (open && elements.isEmpty() && timeoutMillis > 0) { 
-				nonEmpty.await(timeoutMillis, TimeUnit.MILLISECONDS);
-				timeoutMillis = deadline - System.currentTimeMillis();
-			}
-			
-			if (!open) {
-				throw new IllegalStateException("queue is closed");
-			}
-			else if (elements.isEmpty()) {
-				return null;
-			} else {
-				return elements.removeFirst();
-			}
-		} finally {
-			lock.unlock();
-		}
-	}
-
-	/**
-	 * Gets all the elements found in the list, or blocks until at least one element
-	 * was added. If the queue is empty when this method is called, it blocks until
-	 * at least one element is added.
-	 *
-	 * <p>This method always returns a list with at least one element.
-	 * 
-	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
-	 * Checking whether the queue is open and removing the next element is one atomic operation.
-	 * 
-	 * @return A list with all elements in the queue, always at least one element.
-	 * 
-	 * @throws IllegalStateException Thrown, if the queue is closed.
-	 * @throws InterruptedException Throw, if the thread is interrupted while waiting for an
-	 *                              element to be added.
-	 */
-	public List<E> getBatchBlocking() throws InterruptedException {
-		lock.lock();
-		try {
-			while (open && elements.isEmpty()) {
-				nonEmpty.await();
-			}
-			if (open) {
-				ArrayList<E> result = new ArrayList<>(elements);
-				elements.clear();
-				return result;
-			} else {
-				throw new IllegalStateException("queue is closed");
-			}
-		} finally {
-			lock.unlock();
-		}
-	}
-
-	/**
-	 * Gets all the elements found in the list, or blocks until at least one element
-	 * was added. This method is similar as {@link #getBatchBlocking()}, but takes
-	 * a number of milliseconds that the method will maximally wait before returning.
-	 * 
-	 * <p>This method never returns null, but an empty list, if the queue is empty when
-	 * the method is called and the request times out before an element was added.
-	 * 
-	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
-	 * Checking whether the queue is open and removing the next element is one atomic operation.
-	 * 
-	 * @param timeoutMillis The number of milliseconds to wait, at most.
-	 * @return A list with all elements in the queue, possible an empty list.
-	 *
-	 * @throws IllegalStateException Thrown, if the queue is closed.
-	 * @throws InterruptedException Throw, if the thread is interrupted while waiting for an
-	 *                              element to be added.
-	 */
-	public List<E> getBatchBlocking(long timeoutMillis) throws InterruptedException {
-		if (timeoutMillis == 0L) {
-			// wait forever case
-			return getBatchBlocking();
-		} else if (timeoutMillis < 0L) {
-			throw new IllegalArgumentException("invalid timeout");
-		}
-
-		final long deadline = System.currentTimeMillis() + timeoutMillis;
-
-		lock.lock();
-		try {
-			while (open && elements.isEmpty() && timeoutMillis > 0) {
-				nonEmpty.await(timeoutMillis, TimeUnit.MILLISECONDS);
-				timeoutMillis = deadline - System.currentTimeMillis();
-			}
-
-			if (!open) {
-				throw new IllegalStateException("queue is closed");
-			}
-			else if (elements.isEmpty()) {
-				return Collections.emptyList();
-			}
-			else {
-				ArrayList<E> result = new ArrayList<>(elements);
-				elements.clear();
-				return result;
-			}
-		} finally {
-			lock.unlock();
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Standard Utilities
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public int hashCode() {
-		int hashCode = 17;
-		for (E element : elements) {
-			hashCode = 31 * hashCode + element.hashCode();
-		}
-		return hashCode;
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj == this) {
-			return true;
-		} else if (obj != null && obj.getClass() == ClosableBlockingQueue.class) {
-			@SuppressWarnings("unchecked")
-			ClosableBlockingQueue<E> that = (ClosableBlockingQueue<E>) obj;
-			
-			if (this.elements.size() == that.elements.size()) {
-				Iterator<E> thisElements = this.elements.iterator();
-				for (E thatNext : that.elements) {
-					E thisNext = thisElements.next();
-					if (!(thisNext == null ? thatNext == null : thisNext.equals(thatNext))) {
-						return false;
-					}
-				}
-				return true;
-			} else {
-				return false;
-			}
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public String toString() {
-		return elements.toString();
-	}
-}


[13/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml
deleted file mode 100644
index 04019f8..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml
+++ /dev/null
@@ -1,205 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-kafka-0.10_2.10</artifactId>
-	<name>flink-connector-kafka-0.10</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<kafka.version>0.10.0.1</kafka.version>
-	</properties>
-
-	<dependencies>
-
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-0.9_2.10</artifactId>
-			<version>${project.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.apache.kafka</groupId>
-					<artifactId>kafka_${scala.binary.version}</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<!-- Add Kafka 0.10.x as a dependency -->
-
-		<dependency>
-			<groupId>org.apache.kafka</groupId>
-			<artifactId>kafka-clients</artifactId>
-			<version>${kafka.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-			<!-- Projects depending on this project,
-			won't depend on flink-table. -->
-			<optional>true</optional>
-		</dependency>
-
-		<!-- test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-0.9_2.10</artifactId>
-			<version>${project.version}</version>
-			<exclusions>
-				<!-- exclude Kafka dependencies -->
-				<exclusion>
-					<groupId>org.apache.kafka</groupId>
-					<artifactId>kafka_${scala.binary.version}</artifactId>
-				</exclusion>
-			</exclusions>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
-			<version>${project.version}</version>
-			<exclusions>
-				<!-- exclude Kafka dependencies -->
-				<exclusion>
-					<groupId>org.apache.kafka</groupId>
-					<artifactId>kafka_${scala.binary.version}</artifactId>
-				</exclusion>
-			</exclusions>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<!-- include 0.10 server for tests  -->
-			<groupId>org.apache.kafka</groupId>
-			<artifactId>kafka_${scala.binary.version}</artifactId>
-			<version>${kafka.version}</version>
-			<scope>test</scope>
-		</dependency>
-		
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-metrics-jmx</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-						<configuration>
-							<includes>
-								<include>**/KafkaTestEnvironmentImpl*</include>
-							</includes>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-source-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>attach-test-sources</id>
-						<goals>
-							<goal>test-jar-no-fork</goal>
-						</goals>
-						<configuration>
-							<includes>
-								<include>**/KafkaTestEnvironmentImpl*</include>
-							</includes>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
-					<forkCount>1</forkCount>
-					<argLine>-Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-	
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
deleted file mode 100644
index a9ce336..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-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.internal.Kafka010Fetcher;
-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.DeserializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
-import org.apache.flink.util.SerializedValue;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-
-/**
- * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
- * Apache Kafka 0.10.x. The consumer can run in multiple parallel instances, each of which will pull
- * data from one or more Kafka partitions. 
- * 
- * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
- * during a failure, and that the computation processes elements "exactly once". 
- * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
- *
- * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
- * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
- * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
- * has consumed a topic.</p>
- *
- * <p>Please refer to Kafka's documentation for the available configuration properties:
- * http://kafka.apache.org/documentation.html#newconsumerconfigs</p>
- *
- * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
- * is constructed. That means that the client that submits the program needs to be able to
- * reach the Kafka brokers or ZooKeeper.</p>
- */
-public class FlinkKafkaConsumer010<T> extends FlinkKafkaConsumer09<T> {
-
-	private static final long serialVersionUID = 2324564345203409112L;
-
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.10.x
-	 *
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param valueDeserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer010(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		this(Collections.singletonList(topic), valueDeserializer, props);
-	}
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.10.x
-	 *
-	 * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value
-	 * pairs, offsets, and topic names from Kafka.
-	 *
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param deserializer
-	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer010(String topic, KeyedDeserializationSchema<T> deserializer, Properties props) {
-		this(Collections.singletonList(topic), deserializer, props);
-	}
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.10.x
-	 *
-	 * This constructor allows passing multiple topics to the consumer.
-	 *
-	 * @param topics
-	 *           The Kafka topics to read from.
-	 * @param deserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties that are used to configure both the fetcher and the offset handler.
-	 */
-	public FlinkKafkaConsumer010(List<String> topics, DeserializationSchema<T> deserializer, Properties props) {
-		this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props);
-	}
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.10.x
-	 *
-	 * This constructor allows passing multiple topics and a key/value deserialization schema.
-	 *
-	 * @param topics
-	 *           The Kafka topics to read from.
-	 * @param deserializer
-	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties that are used to configure both the fetcher and the offset handler.
-	 */
-	public FlinkKafkaConsumer010(List<String> topics, KeyedDeserializationSchema<T> deserializer, Properties props) {
-		super(topics, deserializer, props);
-	}
-
-	@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 Kafka010Fetcher<>(
-				sourceContext,
-				thisSubtaskPartitions,
-				watermarksPeriodic,
-				watermarksPunctuated,
-				runtimeContext.getProcessingTimeService(),
-				runtimeContext.getExecutionConfig().getAutoWatermarkInterval(),
-				runtimeContext.getUserCodeClassLoader(),
-				runtimeContext.isCheckpointingEnabled(),
-				runtimeContext.getTaskNameWithSubtasks(),
-				runtimeContext.getMetricGroup(),
-				deserializer,
-				properties,
-				pollTimeout,
-				useMetrics);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
deleted file mode 100644
index cc0194b..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
+++ /dev/null
@@ -1,398 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.functions.IterationRuntimeContext;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.kafka.clients.producer.ProducerRecord;
-
-import java.util.Properties;
-
-import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase.getPropertiesFromBrokerList;
-
-
-/**
- * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.10.x
- *
- * Implementation note: This producer is a hybrid between a regular regular sink function (a)
- * and a custom operator (b).
- *
- * For (a), the class implements the SinkFunction and RichFunction interfaces.
- * For (b), it extends the StreamTask class.
- *
- * Details about approach (a):
- *
- *  Pre Kafka 0.10 producers only follow approach (a), allowing users to use the producer using the
- *  DataStream.addSink() method.
- *  Since the APIs exposed in that variant do not allow accessing the the timestamp attached to the record
- *  the Kafka 0.10 producer has a second invocation option, approach (b).
- *
- * Details about approach (b):
- *  Kafka 0.10 supports writing the timestamp attached to a record to Kafka. When adding the
- *  FlinkKafkaProducer010 using the FlinkKafkaProducer010.writeToKafkaWithTimestamps() method, the Kafka producer
- *  can access the internal record timestamp of the record and write it to Kafka.
- *
- * All methods and constructors in this class are marked with the approach they are needed for.
- */
-public class FlinkKafkaProducer010<T> extends StreamSink<T> implements SinkFunction<T>, RichFunction {
-
-	/**
-	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
-	 */
-	private boolean writeTimestampToKafka = false;
-
-	// ---------------------- "Constructors" for timestamp writing ------------------
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
-	 * the topic.
-	 *
-	 * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
-	 *
-	 * @param inStream The stream to write to Kafka
-	 * @param topicId ID of the Kafka topic.
-	 * @param serializationSchema User defined serialization schema supporting key/value messages
-	 * @param producerConfig Properties with the producer configuration.
-	 */
-	public static <T> FlinkKafkaProducer010Configuration<T> writeToKafkaWithTimestamps(DataStream<T> inStream,
-																					String topicId,
-																					KeyedSerializationSchema<T> serializationSchema,
-																					Properties producerConfig) {
-		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FixedPartitioner<T>());
-	}
-
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
-	 * the topic.
-	 *
-	 * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
-	 *
-	 * @param inStream The stream to write to Kafka
-	 * @param topicId ID of the Kafka topic.
-	 * @param serializationSchema User defined (keyless) serialization schema.
-	 * @param producerConfig Properties with the producer configuration.
-	 */
-	public static <T> FlinkKafkaProducer010Configuration<T> writeToKafkaWithTimestamps(DataStream<T> inStream,
-																					String topicId,
-																					SerializationSchema<T> serializationSchema,
-																					Properties producerConfig) {
-		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner<T>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
-	 * the topic.
-	 *
-	 * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
-	 *
-	 *  @param inStream The stream to write to Kafka
-	 *  @param topicId The name of the target topic
-	 *  @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
-	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
-	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
-	 */
-	public static <T> FlinkKafkaProducer010Configuration<T> writeToKafkaWithTimestamps(DataStream<T> inStream,
-																					String topicId,
-																					KeyedSerializationSchema<T> serializationSchema,
-																					Properties producerConfig,
-																					KafkaPartitioner<T> customPartitioner) {
-
-		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
-		FlinkKafkaProducer010<T> kafkaProducer = new FlinkKafkaProducer010<>(topicId, serializationSchema, producerConfig, customPartitioner);
-		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.10.x", objectTypeInfo, kafkaProducer);
-		return new FlinkKafkaProducer010Configuration<>(transformation, kafkaProducer);
-	}
-
-	// ---------------------- Regular constructors w/o timestamp support  ------------------
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 *			Comma separated addresses of the brokers
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined (keyless) serialization schema.
-	 */
-	public FlinkKafkaProducer010(String brokerList, String topicId, SerializationSchema<T> serializationSchema) {
-		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner<T>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined (keyless) serialization schema.
-	 * @param producerConfig
-	 * 			Properties with the producer configuration.
-	 */
-	public FlinkKafkaProducer010(String topicId, SerializationSchema<T> serializationSchema, Properties producerConfig) {
-		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner<T>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param topicId The topic to write data to
-	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
-	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
-	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
-	 */
-	public FlinkKafkaProducer010(String topicId, SerializationSchema<T> serializationSchema, Properties producerConfig, KafkaPartitioner<T> customPartitioner) {
-		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
-	}
-
-	// ------------------- Key/Value serialization schema constructors ----------------------
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 *			Comma separated addresses of the brokers
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined serialization schema supporting key/value messages
-	 */
-	public FlinkKafkaProducer010(String brokerList, String topicId, KeyedSerializationSchema<T> serializationSchema) {
-		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner<T>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined serialization schema supporting key/value messages
-	 * @param producerConfig
-	 * 			Properties with the producer configuration.
-	 */
-	public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema<T> serializationSchema, Properties producerConfig) {
-		this(topicId, serializationSchema, producerConfig, new FixedPartitioner<T>());
-	}
-
-	/**
-	 * Create Kafka producer
-	 *
-	 * This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
-	 */
-	public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema<T> serializationSchema, Properties producerConfig, KafkaPartitioner<T> customPartitioner) {
-		// We create a Kafka 09 producer instance here and only "override" (by intercepting) the
-		// invoke call.
-		super(new FlinkKafkaProducer09<>(topicId, serializationSchema, producerConfig, customPartitioner));
-	}
-
-
-	// ----------------------------- Generic element processing  ---------------------------
-
-	private void invokeInternal(T next, long elementTimestamp) throws Exception {
-
-		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
-
-		internalProducer.checkErroneous();
-
-		byte[] serializedKey = internalProducer.schema.serializeKey(next);
-		byte[] serializedValue = internalProducer.schema.serializeValue(next);
-		String targetTopic = internalProducer.schema.getTargetTopic(next);
-		if (targetTopic == null) {
-			targetTopic = internalProducer.defaultTopicId;
-		}
-
-		Long timestamp = null;
-		if(this.writeTimestampToKafka) {
-			timestamp = elementTimestamp;
-		}
-
-		ProducerRecord<byte[], byte[]> record;
-		if (internalProducer.partitioner == null) {
-			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
-		} else {
-			record = new ProducerRecord<>(targetTopic, internalProducer.partitioner.partition(next, serializedKey, serializedValue, internalProducer.partitions.length), timestamp, serializedKey, serializedValue);
-		}
-		if (internalProducer.flushOnCheckpoint) {
-			synchronized (internalProducer.pendingRecordsLock) {
-				internalProducer.pendingRecords++;
-			}
-		}
-		internalProducer.producer.send(record, internalProducer.callback);
-	}
-
-
-	// ----------------- Helper methods implementing methods from SinkFunction and RichFunction (Approach (a)) ----
-
-
-	// ---- Configuration setters
-
-	/**
-	 * Defines whether the producer should fail on errors, or only log them.
-	 * If this is set to true, then exceptions will be only logged, if set to false,
-	 * exceptions will be eventually thrown and cause the streaming program to
-	 * fail (and enter recovery).
-	 *
-	 * Method is only accessible for approach (a) (see above)
-	 *
-	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
-	 */
-	public void setLogFailuresOnly(boolean logFailuresOnly) {
-		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
-		internalProducer.setLogFailuresOnly(logFailuresOnly);
-	}
-
-	/**
-	 * If set to true, the Flink producer will wait for all outstanding messages in the Kafka buffers
-	 * to be acknowledged by the Kafka producer on a checkpoint.
-	 * This way, the producer can guarantee that messages in the Kafka buffers are part of the checkpoint.
-	 *
-	 * Method is only accessible for approach (a) (see above)
-	 *
-	 * @param flush Flag indicating the flushing mode (true = flush on checkpoint)
-	 */
-	public void setFlushOnCheckpoint(boolean flush) {
-		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
-		internalProducer.setFlushOnCheckpoint(flush);
-	}
-
-	/**
-	 * This method is used for approach (a) (see above)
-	 *
-	 */
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
-		internalProducer.open(parameters);
-	}
-
-	/**
-	 * This method is used for approach (a) (see above)
-	 */
-	@Override
-	public IterationRuntimeContext getIterationRuntimeContext() {
-		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
-		return internalProducer.getIterationRuntimeContext();
-	}
-
-	/**
-	 * This method is used for approach (a) (see above)
-	 */
-	@Override
-	public void setRuntimeContext(RuntimeContext t) {
-		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
-		internalProducer.setRuntimeContext(t);
-	}
-
-	/**
-	 * Invoke method for using the Sink as DataStream.addSink() sink.
-	 *
-	 * This method is used for approach (a) (see above)
-	 *
-	 * @param value The input record.
-	 */
-	@Override
-	public void invoke(T value) throws Exception {
-		invokeInternal(value, Long.MAX_VALUE);
-	}
-
-
-	// ----------------- Helper methods and classes implementing methods from StreamSink (Approach (b)) ----
-
-
-	/**
-	 * Process method for using the sink with timestamp support.
-	 *
-	 * This method is used for approach (b) (see above)
-	 */
-	@Override
-	public void processElement(StreamRecord<T> element) throws Exception {
-		invokeInternal(element.getValue(), element.getTimestamp());
-	}
-
-	/**
-	 * Configuration object returned by the writeToKafkaWithTimestamps() call.
-	 */
-	public static class FlinkKafkaProducer010Configuration<T> extends DataStreamSink<T> {
-
-		private final FlinkKafkaProducerBase wrappedProducerBase;
-		private final FlinkKafkaProducer010 producer;
-
-		private FlinkKafkaProducer010Configuration(DataStream stream, FlinkKafkaProducer010<T> producer) {
-			//noinspection unchecked
-			super(stream, producer);
-			this.producer = producer;
-			this.wrappedProducerBase = (FlinkKafkaProducerBase) producer.userFunction;
-		}
-
-		/**
-		 * Defines whether the producer should fail on errors, or only log them.
-		 * If this is set to true, then exceptions will be only logged, if set to false,
-		 * exceptions will be eventually thrown and cause the streaming program to
-		 * fail (and enter recovery).
-		 *
-		 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
-		 */
-		public void setLogFailuresOnly(boolean logFailuresOnly) {
-			this.wrappedProducerBase.setLogFailuresOnly(logFailuresOnly);
-		}
-
-		/**
-		 * If set to true, the Flink producer will wait for all outstanding messages in the Kafka buffers
-		 * to be acknowledged by the Kafka producer on a checkpoint.
-		 * This way, the producer can guarantee that messages in the Kafka buffers are part of the checkpoint.
-		 *
-		 * @param flush Flag indicating the flushing mode (true = flush on checkpoint)
-		 */
-		public void setFlushOnCheckpoint(boolean flush) {
-			this.wrappedProducerBase.setFlushOnCheckpoint(flush);
-		}
-
-		/**
-		 * If set to true, Flink will write the (event time) timestamp attached to each record into Kafka.
-		 * Timestamps must be positive for Kafka to accept them.
-		 *
-		 * @param writeTimestampToKafka Flag indicating if Flink's internal timestamps are written to Kafka.
-		 */
-		public void setWriteTimestampToKafka(boolean writeTimestampToKafka) {
-			this.producer.writeTimestampToKafka = writeTimestampToKafka;
-		}
-	}
-
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
deleted file mode 100644
index ddf1ad3..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Kafka {@link StreamTableSource} for Kafka 0.10.
- */
-public class Kafka010JsonTableSource extends Kafka09JsonTableSource {
-
-	/**
-	 * Creates a Kafka 0.10 JSON {@link StreamTableSource}.
-	 *
-	 * @param topic      Kafka topic to consume.
-	 * @param properties Properties for the Kafka consumer.
-	 * @param fieldNames Row field names.
-	 * @param fieldTypes Row field types.
-	 */
-	public Kafka010JsonTableSource(
-			String topic,
-			Properties properties,
-			String[] fieldNames,
-			TypeInformation<?>[] fieldTypes) {
-
-		super(topic, properties, fieldNames, fieldTypes);
-	}
-
-	/**
-	 * Creates a Kafka 0.10 JSON {@link StreamTableSource}.
-	 *
-	 * @param topic      Kafka topic to consume.
-	 * @param properties Properties for the Kafka consumer.
-	 * @param fieldNames Row field names.
-	 * @param fieldTypes Row field types.
-	 */
-	public Kafka010JsonTableSource(
-			String topic,
-			Properties properties,
-			String[] fieldNames,
-			Class<?>[] fieldTypes) {
-
-		super(topic, properties, fieldNames, fieldTypes);
-	}
-
-	@Override
-	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
-		return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
deleted file mode 100644
index 732440b..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Kafka {@link StreamTableSource} for Kafka 0.10.
- */
-public class Kafka010TableSource extends Kafka09TableSource {
-
-	/**
-	 * Creates a Kafka 0.10 {@link StreamTableSource}.
-	 *
-	 * @param topic                 Kafka topic to consume.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param deserializationSchema Deserialization schema to use for Kafka records.
-	 * @param fieldNames            Row field names.
-	 * @param fieldTypes            Row field types.
-	 */
-	public Kafka010TableSource(
-			String topic,
-			Properties properties,
-			DeserializationSchema<Row> deserializationSchema,
-			String[] fieldNames,
-			TypeInformation<?>[] fieldTypes) {
-
-		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
-	}
-
-	/**
-	 * Creates a Kafka 0.10 {@link StreamTableSource}.
-	 *
-	 * @param topic                 Kafka topic to consume.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param deserializationSchema Deserialization schema to use for Kafka records.
-	 * @param fieldNames            Row field names.
-	 * @param fieldTypes            Row field types.
-	 */
-	public Kafka010TableSource(
-			String topic,
-			Properties properties,
-			DeserializationSchema<Row> deserializationSchema,
-			String[] fieldNames,
-			Class<?>[] fieldTypes) {
-
-		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
-	}
-
-	@Override
-	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
-		return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java
deleted file mode 100644
index 71dd29a..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internal;
-
-import org.apache.flink.metrics.MetricGroup;
-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.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.util.SerializedValue;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.List;
-import java.util.Properties;
-
-/**
- * A fetcher that fetches data from Kafka brokers via the Kafka 0.10 consumer API.
- * 
- * <p>This fetcher re-uses basically all functionality of the 0.9 fetcher. It only additionally
- * takes the KafkaRecord-attached timestamp and attaches it to the Flink records.
- * 
- * @param <T> The type of elements produced by the fetcher.
- */
-public class Kafka010Fetcher<T> extends Kafka09Fetcher<T> {
-
-	public Kafka010Fetcher(
-			SourceContext<T> sourceContext,
-			List<KafkaTopicPartition> assignedPartitions,
-			SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
-			SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
-			ProcessingTimeService processingTimeProvider,
-			long autoWatermarkInterval,
-			ClassLoader userCodeClassLoader,
-			boolean enableCheckpointing,
-			String taskNameWithSubtasks,
-			MetricGroup metricGroup,
-			KeyedDeserializationSchema<T> deserializer,
-			Properties kafkaProperties,
-			long pollTimeout,
-			boolean useMetrics) throws Exception
-	{
-		super(
-				sourceContext,
-				assignedPartitions,
-				watermarksPeriodic,
-				watermarksPunctuated,
-				processingTimeProvider,
-				autoWatermarkInterval,
-				userCodeClassLoader,
-				enableCheckpointing,
-				taskNameWithSubtasks,
-				metricGroup,
-				deserializer,
-				kafkaProperties,
-				pollTimeout,
-				useMetrics);
-	}
-
-	@Override
-	protected void emitRecord(
-			T record,
-			KafkaTopicPartitionState<TopicPartition> partition,
-			long offset,
-			ConsumerRecord<?, ?> consumerRecord) throws Exception {
-
-		// we attach the Kafka 0.10 timestamp here
-		emitRecordWithTimestamp(record, partition, offset, consumerRecord.timestamp());
-	}
-
-	/**
-	 * This method needs to be overridden because Kafka broke binary compatibility between 0.9 and 0.10,
-	 * changing binary signatures
-	 */
-	@Override
-	protected KafkaConsumerCallBridge010 createCallBridge() {
-		return new KafkaConsumerCallBridge010();
-	}
-
-	@Override
-	protected String getFetcherName() {
-		return "Kafka 0.10 Fetcher";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java
deleted file mode 100644
index a81b098..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internal;
-
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.List;
-
-/**
- * The ConsumerCallBridge simply calls the {@link KafkaConsumer#assign(java.util.Collection)} method.
- * 
- * This indirection is necessary, because Kafka broke binary compatibility between 0.9 and 0.10,
- * changing {@code assign(List)} to {@code assign(Collection)}.
- * 
- * Because of that, we need two versions whose compiled code goes against different method signatures.
- */
-public class KafkaConsumerCallBridge010 extends KafkaConsumerCallBridge {
-
-	@Override
-	public void assignPartitions(KafkaConsumer<?, ?> consumer, List<TopicPartition> topicPartitions) throws Exception {
-		consumer.assign(topicPartitions);
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java
deleted file mode 100644
index 6ee0429..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java
+++ /dev/null
@@ -1,484 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.core.testutils.MultiShotLatch;
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
-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.connectors.kafka.internal.Handover;
-import org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher;
-import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-
-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.clients.consumer.OffsetCommitCallback;
-import org.apache.kafka.common.TopicPartition;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import org.mockito.Mockito;
-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.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantLock;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.anyLong;
-import static org.powermock.api.mockito.PowerMockito.doAnswer;
-import static org.powermock.api.mockito.PowerMockito.mock;
-import static org.powermock.api.mockito.PowerMockito.when;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
-/**
- * Unit tests for the {@link Kafka010Fetcher}.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(KafkaConsumerThread.class)
-public class Kafka010FetcherTest {
-
-    @Test
-    public void testCommitDoesNotBlock() throws Exception {
-
-        // test data
-        final KafkaTopicPartition testPartition = new KafkaTopicPartition("test", 42);
-        final Map<KafkaTopicPartition, Long> testCommitData = new HashMap<>();
-        testCommitData.put(testPartition, 11L);
-
-        // to synchronize when the consumer is in its blocking method
-        final OneShotLatch sync = new OneShotLatch();
-
-        // ----- the mock consumer with blocking poll calls ----
-        final MultiShotLatch blockerLatch = new MultiShotLatch();
-
-        KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
-        when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
-
-            @Override
-            public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) throws InterruptedException {
-                sync.trigger();
-                blockerLatch.await();
-                return ConsumerRecords.empty();
-            }
-        });
-
-        doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) {
-                blockerLatch.trigger();
-                return null;
-            }
-        }).when(mockConsumer).wakeup();
-
-        // make sure the fetcher creates the mock consumer
-        whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
-
-        // ----- create the test fetcher -----
-
-        @SuppressWarnings("unchecked")
-        SourceContext<String> sourceContext = mock(SourceContext.class);
-        List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition("test", 42));
-        KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
-
-        final Kafka010Fetcher<String> fetcher = new Kafka010Fetcher<>(
-                sourceContext,
-                topics,
-                null, /* periodic assigner */
-                null, /* punctuated assigner */
-                new TestProcessingTimeService(),
-                10,
-                getClass().getClassLoader(),
-                false, /* checkpointing */
-                "taskname-with-subtask",
-                new UnregisteredMetricsGroup(),
-                schema,
-                new Properties(),
-                0L,
-                false);
-
-        // ----- run the fetcher -----
-
-        final AtomicReference<Throwable> error = new AtomicReference<>();
-        final Thread fetcherRunner = new Thread("fetcher runner") {
-
-            @Override
-            public void run() {
-                try {
-                    fetcher.runFetchLoop();
-                } catch (Throwable t) {
-                    error.set(t);
-                }
-            }
-        };
-        fetcherRunner.start();
-
-        // wait until the fetcher has reached the method of interest
-        sync.await();
-
-        // ----- trigger the offset commit -----
-
-        final AtomicReference<Throwable> commitError = new AtomicReference<>();
-        final Thread committer = new Thread("committer runner") {
-            @Override
-            public void run() {
-                try {
-                    fetcher.commitInternalOffsetsToKafka(testCommitData);
-                } catch (Throwable t) {
-                    commitError.set(t);
-                }
-            }
-        };
-        committer.start();
-
-        // ----- ensure that the committer finishes in time  -----
-        committer.join(30000);
-        assertFalse("The committer did not finish in time", committer.isAlive());
-
-        // ----- test done, wait till the fetcher is done for a clean shutdown -----
-        fetcher.cancel();
-        fetcherRunner.join();
-
-        // check that there were no errors in the fetcher
-        final Throwable fetcherError = error.get();
-        if (fetcherError != null && !(fetcherError instanceof Handover.ClosedException)) {
-            throw new Exception("Exception in the fetcher", fetcherError);
-        }
-        final Throwable committerError = commitError.get();
-        if (committerError != null) {
-            throw new Exception("Exception in the committer", committerError);
-        }
-    }
-
-    @Test
-    public void ensureOffsetsGetCommitted() throws Exception {
-
-        // test data
-        final KafkaTopicPartition testPartition1 = new KafkaTopicPartition("test", 42);
-        final KafkaTopicPartition testPartition2 = new KafkaTopicPartition("another", 99);
-
-        final Map<KafkaTopicPartition, Long> testCommitData1 = new HashMap<>();
-        testCommitData1.put(testPartition1, 11L);
-        testCommitData1.put(testPartition2, 18L);
-
-        final Map<KafkaTopicPartition, Long> testCommitData2 = new HashMap<>();
-        testCommitData2.put(testPartition1, 19L);
-        testCommitData2.put(testPartition2, 28L);
-
-        final BlockingQueue<Map<TopicPartition, OffsetAndMetadata>> commitStore = new LinkedBlockingQueue<>();
-
-
-        // ----- the mock consumer with poll(), wakeup(), and commit(A)sync calls ----
-
-        final MultiShotLatch blockerLatch = new MultiShotLatch();
-
-        KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
-
-        when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
-            @Override
-            public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) throws InterruptedException {
-                blockerLatch.await();
-                return ConsumerRecords.empty();
-            }
-        });
-
-        doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) {
-                blockerLatch.trigger();
-                return null;
-            }
-        }).when(mockConsumer).wakeup();
-
-        doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) {
-                @SuppressWarnings("unchecked")
-                Map<TopicPartition, OffsetAndMetadata> offsets =
-                        (Map<TopicPartition, OffsetAndMetadata>) invocation.getArguments()[0];
-
-                OffsetCommitCallback callback = (OffsetCommitCallback) invocation.getArguments()[1];
-
-                commitStore.add(offsets);
-                callback.onComplete(offsets, null);
-
-                return null;
-            }
-        }).when(mockConsumer).commitAsync(
-                Mockito.<Map<TopicPartition, OffsetAndMetadata>>any(), any(OffsetCommitCallback.class));
-
-        // make sure the fetcher creates the mock consumer
-        whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
-
-        // ----- create the test fetcher -----
-
-        @SuppressWarnings("unchecked")
-        SourceContext<String> sourceContext = mock(SourceContext.class);
-        List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition("test", 42));
-        KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
-        StreamingRuntimeContext context = mock(StreamingRuntimeContext.class);
-
-        final Kafka010Fetcher<String> fetcher = new Kafka010Fetcher<>(
-                sourceContext,
-                topics,
-                null, /* periodic assigner */
-                null, /* punctuated assigner */
-                new TestProcessingTimeService(),
-                10,
-                getClass().getClassLoader(),
-                false, /* checkpointing */
-                "taskname-with-subtask",
-                new UnregisteredMetricsGroup(),
-                schema,
-                new Properties(),
-                0L,
-                false);
-
-
-        // ----- run the fetcher -----
-
-        final AtomicReference<Throwable> error = new AtomicReference<>();
-        final Thread fetcherRunner = new Thread("fetcher runner") {
-
-            @Override
-            public void run() {
-                try {
-                    fetcher.runFetchLoop();
-                } catch (Throwable t) {
-                    error.set(t);
-                }
-            }
-        };
-        fetcherRunner.start();
-
-        // ----- trigger the first offset commit -----
-
-        fetcher.commitInternalOffsetsToKafka(testCommitData1);
-        Map<TopicPartition, OffsetAndMetadata> result1 = commitStore.take();
-
-        for (Entry<TopicPartition, OffsetAndMetadata> entry : result1.entrySet()) {
-            TopicPartition partition = entry.getKey();
-            if (partition.topic().equals("test")) {
-                assertEquals(42, partition.partition());
-                assertEquals(12L, entry.getValue().offset());
-            }
-            else if (partition.topic().equals("another")) {
-                assertEquals(99, partition.partition());
-                assertEquals(18L, entry.getValue().offset());
-            }
-        }
-
-        // ----- trigger the second offset commit -----
-
-        fetcher.commitInternalOffsetsToKafka(testCommitData2);
-        Map<TopicPartition, OffsetAndMetadata> result2 = commitStore.take();
-
-        for (Entry<TopicPartition, OffsetAndMetadata> entry : result2.entrySet()) {
-            TopicPartition partition = entry.getKey();
-            if (partition.topic().equals("test")) {
-                assertEquals(42, partition.partition());
-                assertEquals(20L, entry.getValue().offset());
-            }
-            else if (partition.topic().equals("another")) {
-                assertEquals(99, partition.partition());
-                assertEquals(28L, entry.getValue().offset());
-            }
-        }
-
-        // ----- test done, wait till the fetcher is done for a clean shutdown -----
-        fetcher.cancel();
-        fetcherRunner.join();
-
-        // check that there were no errors in the fetcher
-        final Throwable caughtError = error.get();
-        if (caughtError != null && !(caughtError instanceof Handover.ClosedException)) {
-            throw new Exception("Exception in the fetcher", caughtError);
-        }
-    }
-
-    @Test
-    public void testCancellationWhenEmitBlocks() throws Exception {
-
-        // ----- some test data -----
-
-        final String topic = "test-topic";
-        final int partition = 3;
-        final byte[] payload = new byte[] {1, 2, 3, 4};
-
-        final List<ConsumerRecord<byte[], byte[]>> records = Arrays.asList(
-                new ConsumerRecord<byte[], byte[]>(topic, partition, 15, payload, payload),
-                new ConsumerRecord<byte[], byte[]>(topic, partition, 16, payload, payload),
-                new ConsumerRecord<byte[], byte[]>(topic, partition, 17, payload, payload));
-
-        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> data = new HashMap<>();
-        data.put(new TopicPartition(topic, partition), records);
-
-        final ConsumerRecords<byte[], byte[]> consumerRecords = new ConsumerRecords<>(data);
-
-        // ----- the test consumer -----
-
-        final KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
-        when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
-            @Override
-            public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) {
-                return consumerRecords;
-            }
-        });
-
-        whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
-
-        // ----- build a fetcher -----
-
-        BlockingSourceContext<String> sourceContext = new BlockingSourceContext<>();
-        List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition(topic, partition));
-        KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
-
-        final Kafka010Fetcher<String> fetcher = new Kafka010Fetcher<>(
-                sourceContext,
-                topics,
-                null, /* periodic watermark extractor */
-                null, /* punctuated watermark extractor */
-                new TestProcessingTimeService(),
-                10, /* watermark interval */
-                this.getClass().getClassLoader(),
-                true, /* checkpointing */
-                "task_name",
-                new UnregisteredMetricsGroup(),
-                schema,
-                new Properties(),
-                0L,
-                false);
-
-
-        // ----- run the fetcher -----
-
-        final AtomicReference<Throwable> error = new AtomicReference<>();
-        final Thread fetcherRunner = new Thread("fetcher runner") {
-
-            @Override
-            public void run() {
-                try {
-                    fetcher.runFetchLoop();
-                } catch (Throwable t) {
-                    error.set(t);
-                }
-            }
-        };
-        fetcherRunner.start();
-
-        // wait until the thread started to emit records to the source context
-        sourceContext.waitTillHasBlocker();
-
-        // now we try to cancel the fetcher, including the interruption usually done on the task thread
-        // once it has finished, there must be no more thread blocked on the source context
-        fetcher.cancel();
-        fetcherRunner.interrupt();
-        fetcherRunner.join();
-
-        assertFalse("fetcher threads did not properly finish", sourceContext.isStillBlocking());
-    }
-
-    // ------------------------------------------------------------------------
-    //  test utilities
-    // ------------------------------------------------------------------------
-
-    private static final class BlockingSourceContext<T> implements SourceContext<T> {
-
-        private final ReentrantLock lock = new ReentrantLock();
-        private final OneShotLatch inBlocking = new OneShotLatch();
-
-        @Override
-        public void collect(T element) {
-            block();
-        }
-
-        @Override
-        public void collectWithTimestamp(T element, long timestamp) {
-            block();
-        }
-
-        @Override
-        public void emitWatermark(Watermark mark) {
-            block();
-        }
-
-        @Override
-        public Object getCheckpointLock() {
-            return new Object();
-        }
-
-        @Override
-        public void close() {}
-
-        public void waitTillHasBlocker() throws InterruptedException {
-            inBlocking.await();
-        }
-
-        public boolean isStillBlocking() {
-            return lock.isLocked();
-        }
-
-        @SuppressWarnings({"InfiniteLoopStatement", "SynchronizationOnLocalVariableOrMethodParameter"})
-        private void block() {
-            lock.lock();
-            try {
-                inBlocking.trigger();
-
-                // put this thread to sleep indefinitely
-                final Object o = new Object();
-                while (true) {
-                    synchronized (o) {
-                        o.wait();
-                    }
-                }
-            }
-            catch (InterruptedException e) {
-                // exit cleanly, simply reset the interruption flag
-                Thread.currentThread().interrupt();
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
deleted file mode 100644
index 08511c9..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
+++ /dev/null
@@ -1,313 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
-import org.junit.Test;
-
-import javax.annotation.Nullable;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-
-
-public class Kafka010ITCase extends KafkaConsumerTestBase {
-
-	// ------------------------------------------------------------------------
-	//  Suite of Tests
-	// ------------------------------------------------------------------------
-
-
-	@Test(timeout = 60000)
-	public void testFailOnNoBroker() throws Exception {
-		runFailOnNoBrokerTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testConcurrentProducerConsumerTopology() throws Exception {
-		runSimpleConcurrentProducerConsumerTopology();
-	}
-
-	@Test(timeout = 60000)
-	public void testKeyValueSupport() throws Exception {
-		runKeyValueTest();
-	}
-
-	// --- canceling / failures ---
-
-	@Test(timeout = 60000)
-	public void testCancelingEmptyTopic() throws Exception {
-		runCancelingOnEmptyInputTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testCancelingFullTopic() throws Exception {
-		runCancelingOnFullInputTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testFailOnDeploy() throws Exception {
-		runFailOnDeployTest();
-	}
-
-
-	// --- source to partition mappings and exactly once ---
-
-	@Test(timeout = 60000)
-	public void testOneToOneSources() throws Exception {
-		runOneToOneExactlyOnceTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testOneSourceMultiplePartitions() throws Exception {
-		runOneSourceMultiplePartitionsExactlyOnceTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testMultipleSourcesOnePartition() throws Exception {
-		runMultipleSourcesOnePartitionExactlyOnceTest();
-	}
-
-	// --- broker failure ---
-
-	@Test(timeout = 60000)
-	public void testBrokerFailure() throws Exception {
-		runBrokerFailureTest();
-	}
-
-	// --- special executions ---
-
-	@Test(timeout = 60000)
-	public void testBigRecordJob() throws Exception {
-		runBigRecordTestTopology();
-	}
-
-	@Test(timeout = 60000)
-	public void testMultipleTopics() throws Exception {
-		runProduceConsumeMultipleTopics();
-	}
-
-	@Test(timeout = 60000)
-	public void testAllDeletes() throws Exception {
-		runAllDeletesTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testMetricsAndEndOfStream() throws Exception {
-		runEndOfStreamTest();
-	}
-
-	// --- offset committing ---
-
-	@Test(timeout = 60000)
-	public void testCommitOffsetsToKafka() throws Exception {
-		runCommitOffsetsToKafka();
-	}
-
-	@Test(timeout = 60000)
-	public void testStartFromKafkaCommitOffsets() throws Exception {
-		runStartFromKafkaCommitOffsets();
-	}
-
-	@Test(timeout = 60000)
-	public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception {
-		runAutoOffsetRetrievalAndCommitToKafka();
-	}
-
-	/**
-	 * Kafka 0.10 specific test, ensuring Timestamps are properly written to and read from Kafka
-	 */
-	@Test(timeout = 60000)
-	public void testTimestamps() throws Exception {
-
-		final String topic = "tstopic";
-		createTestTopic(topic, 3, 1);
-
-		// ---------- Produce an event time stream into Kafka -------------------
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(1);
-		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env.getConfig().disableSysoutLogging();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-
-		DataStream<Long> streamWithTimestamps = env.addSource(new SourceFunction<Long>() {
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<Long> ctx) throws Exception {
-				long i = 0;
-				while(running) {
-					ctx.collectWithTimestamp(i, i*2);
-					if(i++ == 1000L) {
-						running = false;
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-				running = false;
-			}
-		});
-
-		final TypeInformationSerializationSchema<Long> longSer = new TypeInformationSerializationSchema<>(TypeInfoParser.<Long>parse("Long"), env.getConfig());
-		FlinkKafkaProducer010.FlinkKafkaProducer010Configuration prod = FlinkKafkaProducer010.writeToKafkaWithTimestamps(streamWithTimestamps, topic, new KeyedSerializationSchemaWrapper<>(longSer), standardProps, new KafkaPartitioner<Long>() {
-			@Override
-			public int partition(Long next, byte[] serializedKey, byte[] serializedValue, int numPartitions) {
-				return (int)(next % 3);
-			}
-		});
-		prod.setParallelism(3);
-		prod.setWriteTimestampToKafka(true);
-		env.execute("Produce some");
-
-		// ---------- Consume stream from Kafka -------------------
-
-		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(1);
-		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env.getConfig().disableSysoutLogging();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-
-		FlinkKafkaConsumer010<Long> kafkaSource = new FlinkKafkaConsumer010<>(topic, new LimitedLongDeserializer(), standardProps);
-		kafkaSource.assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks<Long>() {
-			@Nullable
-			@Override
-			public Watermark checkAndGetNextWatermark(Long lastElement, long extractedTimestamp) {
-				if(lastElement % 10 == 0) {
-					return new Watermark(lastElement);
-				}
-				return null;
-			}
-
-			@Override
-			public long extractTimestamp(Long element, long previousElementTimestamp) {
-				return previousElementTimestamp;
-			}
-		});
-
-		DataStream<Long> stream = env.addSource(kafkaSource);
-		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
-		stream.transform("timestamp validating operator", objectTypeInfo, new TimestampValidatingOperator()).setParallelism(1);
-
-		env.execute("Consume again");
-
-		deleteTestTopic(topic);
-	}
-
-	private static class TimestampValidatingOperator extends StreamSink<Long> {
-
-		public TimestampValidatingOperator() {
-			super(new SinkFunction<Long>() {
-				@Override
-				public void invoke(Long value) throws Exception {
-					throw new RuntimeException("Unexpected");
-				}
-			});
-		}
-
-		long elCount = 0;
-		long wmCount = 0;
-		long lastWM = Long.MIN_VALUE;
-
-		@Override
-		public void processElement(StreamRecord<Long> element) throws Exception {
-			elCount++;
-			if(element.getValue() * 2 != element.getTimestamp()) {
-				throw new RuntimeException("Invalid timestamp: " + element);
-			}
-		}
-
-		@Override
-		public void processWatermark(Watermark mark) throws Exception {
-			wmCount++;
-
-			if(lastWM <= mark.getTimestamp()) {
-				lastWM = mark.getTimestamp();
-			} else {
-				throw new RuntimeException("Received watermark higher than the last one");
-			}
-
-			if( mark.getTimestamp() % 10 != 0 && mark.getTimestamp() != Long.MAX_VALUE ) {
-				throw new RuntimeException("Invalid watermark: " + mark.getTimestamp());
-			}
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if(elCount != 1000L) {
-				throw new RuntimeException("Wrong final element count " + elCount);
-			}
-
-			if(wmCount <= 2) {
-				throw new RuntimeException("Almost no watermarks have been sent " + wmCount);
-			}
-		}
-	}
-
-	private static class LimitedLongDeserializer implements KeyedDeserializationSchema<Long> {
-
-		private final TypeInformation<Long> ti;
-		private final TypeSerializer<Long> ser;
-		long cnt = 0;
-
-		public LimitedLongDeserializer() {
-			this.ti = TypeInfoParser.parse("Long");
-			this.ser = ti.createSerializer(new ExecutionConfig());
-		}
-		@Override
-		public TypeInformation<Long> getProducedType() {
-			return ti;
-		}
-
-		@Override
-		public Long deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
-			cnt++;
-			DataInputView in = new DataInputViewStreamWrapper(new ByteArrayInputStream(message));
-			Long e = ser.deserialize(in);
-			return e;
-		}
-
-		@Override
-		public boolean isEndOfStream(Long nextElement) {
-			return cnt > 1000L;
-		}
-	}
-
-}


[42/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java
new file mode 100644
index 0000000..650931f
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.elasticsearch2;
+
+import org.elasticsearch.action.ActionRequest;
+import org.elasticsearch.action.bulk.BulkProcessor;
+
+public class BulkProcessorIndexer implements RequestIndexer {
+	private final BulkProcessor bulkProcessor;
+
+	public BulkProcessorIndexer(BulkProcessor bulkProcessor) {
+		this.bulkProcessor = bulkProcessor;
+	}
+
+	@Override
+	public void add(ActionRequest... actionRequests) {
+		for (ActionRequest actionRequest : actionRequests) {
+			this.bulkProcessor.add(actionRequest);
+		}
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java
new file mode 100644
index 0000000..55ba720
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.elasticsearch2;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import java.io.Serializable;
+
+/**
+ * Method that creates an {@link org.elasticsearch.action.ActionRequest} from an element in a Stream.
+ *
+ * <p>
+ * This is used by {@link ElasticsearchSink} to prepare elements for sending them to Elasticsearch.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre>{@code
+ *					private static class TestElasticSearchSinkFunction implements
+ *						ElasticsearchSinkFunction<Tuple2<Integer, String>> {
+ *
+ *					public IndexRequest createIndexRequest(Tuple2<Integer, String> element) {
+ *						Map<String, Object> json = new HashMap<>();
+ *						json.put("data", element.f1);
+ *
+ *						return Requests.indexRequest()
+ *							.index("my-index")
+ *							.type("my-type")
+ *							.id(element.f0.toString())
+ *							.source(json);
+ *						}
+ *
+ *				public void process(Tuple2<Integer, String> element, RuntimeContext ctx, RequestIndexer indexer) {
+ *					indexer.add(createIndexRequest(element));
+ *				}
+ *		}
+ *
+ * }</pre>
+ *
+ * @param <T> The type of the element handled by this {@code ElasticsearchSinkFunction}
+ */
+public interface ElasticsearchSinkFunction<T> extends Serializable, Function {
+	void process(T element, RuntimeContext ctx, RequestIndexer indexer);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java
new file mode 100644
index 0000000..144a87b
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.elasticsearch2;
+
+import org.elasticsearch.action.ActionRequest;
+
+import java.io.Serializable;
+
+public interface RequestIndexer extends Serializable {
+	void add(ActionRequest... actionRequests);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
new file mode 100644
index 0000000..bc9bedc
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
@@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.elasticsearch2;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.elasticsearch.action.get.GetRequest;
+import org.elasticsearch.action.get.GetResponse;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.Requests;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.node.NodeBuilder;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase {
+
+	private static final int NUM_ELEMENTS = 20;
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Test
+	public void testTransportClient() throws Exception {
+
+		File dataDir = tempFolder.newFolder();
+
+		Node node = NodeBuilder.nodeBuilder()
+				.settings(Settings.settingsBuilder()
+						.put("path.home", dataDir.getParent())
+						.put("http.enabled", false)
+						.put("path.data", dataDir.getAbsolutePath()))
+				// set a custom cluster name to verify that user config works correctly
+				.clusterName("my-transport-client-cluster")
+				.node();
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
+
+		Map<String, String> config = new HashMap<>();
+		// This instructs the sink to emit after every element, otherwise they would be buffered
+		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+		config.put("cluster.name", "my-transport-client-cluster");
+
+		// Can't use {@link TransportAddress} as its not Serializable in Elasticsearch 2.x
+		List<InetSocketAddress> transports = new ArrayList<>();
+		transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
+
+		source.addSink(new ElasticsearchSink<>(config, transports, new TestElasticsearchSinkFunction()));
+
+		env.execute("Elasticsearch TransportClient Test");
+
+		// verify the results
+		Client client = node.client();
+		for (int i = 0; i < NUM_ELEMENTS; i++) {
+			GetResponse response = client.get(new GetRequest("my-index",
+					"my-type", Integer.toString(i))).actionGet();
+			Assert.assertEquals("message #" + i, response.getSource().get("data"));
+		}
+
+		node.close();
+	}
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testNullTransportClient() throws Exception {
+
+	File dataDir = tempFolder.newFolder();
+
+	Node node = NodeBuilder.nodeBuilder()
+		.settings(Settings.settingsBuilder()
+			.put("path.home", dataDir.getParent())
+			.put("http.enabled", false)
+			.put("path.data", dataDir.getAbsolutePath()))
+		// set a custom cluster name to verify that user config works correctly
+		.clusterName("my-transport-client-cluster")
+		.node();
+
+	final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+	DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
+
+	Map<String, String> config = new HashMap<>();
+	// This instructs the sink to emit after every element, otherwise they would be buffered
+	config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+	config.put("cluster.name", "my-transport-client-cluster");
+
+	source.addSink(new ElasticsearchSink<>(config, null, new TestElasticsearchSinkFunction()));
+
+	env.execute("Elasticsearch TransportClient Test");
+
+	// verify the results
+	Client client = node.client();
+	for (int i = 0; i < NUM_ELEMENTS; i++) {
+	 GetResponse response = client.get(new GetRequest("my-index",
+		 "my-type", Integer.toString(i))).actionGet();
+	 Assert.assertEquals("message #" + i, response.getSource().get("data"));
+	}
+
+	node.close();
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testEmptyTransportClient() throws Exception {
+
+	File dataDir = tempFolder.newFolder();
+
+	Node node = NodeBuilder.nodeBuilder()
+		.settings(Settings.settingsBuilder()
+			.put("path.home", dataDir.getParent())
+			.put("http.enabled", false)
+			.put("path.data", dataDir.getAbsolutePath()))
+		// set a custom cluster name to verify that user config works correctly
+		.clusterName("my-transport-client-cluster")
+		.node();
+
+	final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+	DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
+
+	Map<String, String> config = new HashMap<>();
+	// This instructs the sink to emit after every element, otherwise they would be buffered
+	config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+	config.put("cluster.name", "my-transport-client-cluster");
+
+	source.addSink(new ElasticsearchSink<>(config, new ArrayList<InetSocketAddress>(), new TestElasticsearchSinkFunction()));
+
+	env.execute("Elasticsearch TransportClient Test");
+
+	// verify the results
+	Client client = node.client();
+	for (int i = 0; i < NUM_ELEMENTS; i++) {
+	 GetResponse response = client.get(new GetRequest("my-index",
+		 "my-type", Integer.toString(i))).actionGet();
+	 Assert.assertEquals("message #" + i, response.getSource().get("data"));
+	}
+
+	node.close();
+ }
+
+	@Test(expected = JobExecutionException.class)
+	public void testTransportClientFails() throws Exception{
+		// this checks whether the TransportClient fails early when there is no cluster to
+		// connect to. There isn't a similar test for the Node Client version since that
+		// one will block and wait for a cluster to come online
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
+
+		Map<String, String> config = new HashMap<>();
+		// This instructs the sink to emit after every element, otherwise they would be buffered
+		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+		config.put("cluster.name", "my-node-client-cluster");
+
+		List<InetSocketAddress> transports = new ArrayList<>();
+		transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
+
+		source.addSink(new ElasticsearchSink<>(config, transports, new TestElasticsearchSinkFunction()));
+
+		env.execute("Elasticsearch Node Client Test");
+	}
+
+	private static class TestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		@Override
+		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
+			for (int i = 0; i < NUM_ELEMENTS && running; i++) {
+				ctx.collect(Tuple2.of(i, "message #" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+	private static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		public IndexRequest createIndexRequest(Tuple2<Integer, String> element) {
+			Map<String, Object> json = new HashMap<>();
+			json.put("data", element.f1);
+
+			return Requests.indexRequest()
+					.index("my-index")
+					.type("my-type")
+					.id(element.f0.toString())
+					.source(json);
+		}
+
+		@Override
+		public void process(Tuple2<Integer, String> element, RuntimeContext ctx, RequestIndexer indexer) {
+			indexer.add(createIndexRequest(element));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java
new file mode 100644
index 0000000..05760e8
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.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.elasticsearch2.examples;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink;
+import org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSinkFunction;
+import org.apache.flink.streaming.connectors.elasticsearch2.RequestIndexer;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.client.Requests;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This example shows how to use the Elasticsearch Sink. Before running it you must ensure that
+ * you have a cluster named "elasticsearch" running or change the name of cluster in the config map.
+ */
+public class ElasticsearchExample {
+
+	public static void main(String[] args) throws Exception {
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		SingleOutputStreamOperator<String> source =
+				env.generateSequence(0, 20).map(new MapFunction<Long, String>() {
+					/**
+					 * The mapping method. Takes an element from the input data set and transforms
+					 * it into exactly one element.
+					 *
+					 * @param value The input value.
+					 * @return The transformed value
+					 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
+					 *                   to fail and may trigger recovery.
+					 */
+					@Override
+					public String map(Long value) throws Exception {
+						return "message #" + value;
+					}
+				});
+
+		Map<String, String> config = new HashMap<>();
+		// This instructs the sink to emit after every element, otherwise they would be buffered
+		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+
+		List<InetSocketAddress> transports = new ArrayList<>();
+		transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
+
+		source.addSink(new ElasticsearchSink<>(config, transports, new ElasticsearchSinkFunction<String>(){
+			@Override
+			public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
+				indexer.add(createIndexRequest(element));
+			}
+		}));
+
+		env.execute("Elasticsearch Example");
+	}
+
+	private static IndexRequest createIndexRequest(String element) {
+		Map<String, Object> json = new HashMap<>();
+		json.put("data", element);
+
+		return Requests.indexRequest()
+				.index("my-index")
+				.type("my-type")
+				.id(element)
+				.source(json);
+	}
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/pom.xml b/flink-connectors/flink-connector-filesystem/pom.xml
new file mode 100644
index 0000000..fbc830a
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/pom.xml
@@ -0,0 +1,163 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-filesystem_2.10</artifactId>
+	<name>flink-connector-filesystem</name>
+
+	<packaging>jar</packaging>
+
+	<!--
+		This is a Hadoop2 only flink module.
+	-->
+
+	<dependencies>
+
+		<!-- core dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-hadoop2</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+		
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<scope>test</scope>
+			<type>test-jar</type>
+			<version>${hadoop.version}</version><!--$NO-MVN-MAN-VER$-->
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>test</scope>
+			<type>test-jar</type>
+			<version>${hadoop.version}</version><!--$NO-MVN-MAN-VER$-->
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-minikdc</artifactId>
+			<version>${minikdc.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+	</dependencies>
+
+	<build>
+		<plugins>
+
+			<!--
+				https://issues.apache.org/jira/browse/DIRSHARED-134
+				Required to pull the Mini-KDC transitive dependency
+			-->
+			<plugin>
+				<groupId>org.apache.felix</groupId>
+				<artifactId>maven-bundle-plugin</artifactId>
+				<version>3.0.1</version>
+				<inherited>true</inherited>
+				<extensions>true</extensions>
+			</plugin>
+
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<!--
+					Enforce single threaded execution to avoid port conflicts when running
+					secure mini DFS cluster
+					-->
+					<forkCount>1</forkCount>
+					<reuseForks>false</reuseForks>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
new file mode 100644
index 0000000..3e3c86b
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
@@ -0,0 +1,309 @@
+package org.apache.flink.streaming.connectors.fs;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.CodecFactory;
+import org.apache.avro.file.DataFileConstants;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+/**
+* Implementation of AvroKeyValue writer that can be used in Sink.
+* Each entry would be wrapped in GenericRecord with key/value fields(same as in m/r lib)
+<pre>
+Usage:
+{@code
+		BucketingSink<Tuple2<Long, Long>> sink = new BucketingSink<Tuple2<Long, Long>>("/tmp/path");
+		sink.setBucketer(new DateTimeBucketer<Tuple2<Long, Long>>("yyyy-MM-dd/HH/mm/"));
+		sink.setPendingSuffix(".avro");
+		Map<String,String> properties = new HashMap<>();
+		Schema longSchema = Schema.create(Type.LONG);
+		String keySchema = longSchema.toString();
+		String valueSchema = longSchema.toString();
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema);
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema);
+		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS, Boolean.toString(true));
+		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS_CODEC, DataFileConstants.SNAPPY_CODEC);
+		
+		sink.setWriter(new AvroSinkWriter<Long, Long>(properties));
+		sink.setBatchSize(1024 * 1024 * 64); // this is 64 MB,
+}
+</pre>
+*/
+public class AvroKeyValueSinkWriter<K, V> extends StreamWriterBase<Tuple2<K, V>>  implements Writer<Tuple2<K, V>>, InputTypeConfigurable {
+	private static final long serialVersionUID = 1L;
+	public static final String CONF_OUTPUT_KEY_SCHEMA = "avro.schema.output.key";
+	public static final String CONF_OUTPUT_VALUE_SCHEMA = "avro.schema.output.value";
+	public static final String CONF_COMPRESS = FileOutputFormat.COMPRESS;
+	public static final String CONF_COMPRESS_CODEC = FileOutputFormat.COMPRESS_CODEC;
+	public static final String CONF_DEFLATE_LEVEL = "avro.deflate.level";
+	public static final String CONF_XZ_LEVEL = "avro.xz.level";
+
+	private transient AvroKeyValueWriter<K, V> keyValueWriter;
+
+	private final Map<String, String> properties;
+
+	/**
+	 * C'tor for the writer
+	 * <p>
+	 * You can provide different properties that will be used to configure avro key-value writer as simple properties map(see example above)
+	 * @param properties
+	 */
+	@SuppressWarnings("deprecation")
+	public AvroKeyValueSinkWriter(Map<String, String> properties) {
+		this.properties = properties;
+		
+		String keySchemaString = properties.get(CONF_OUTPUT_KEY_SCHEMA);
+		if (keySchemaString == null) {
+			throw new IllegalStateException("No key schema provided, set '" + CONF_OUTPUT_KEY_SCHEMA + "' property");
+		}
+		Schema.parse(keySchemaString);//verifying that schema valid
+		
+		String valueSchemaString = properties.get(CONF_OUTPUT_VALUE_SCHEMA);
+		if (valueSchemaString == null) {
+			throw new IllegalStateException("No value schema provided, set '" + CONF_OUTPUT_VALUE_SCHEMA + "' property");
+		}
+		Schema.parse(valueSchemaString);//verifying that schema valid
+	}
+
+	private boolean getBoolean(Map<String,String> conf, String key, boolean def) {
+		String value = conf.get(key);
+		if (value == null) {
+			return def;
+		}
+		return Boolean.parseBoolean(value);
+	}
+	
+	private int getInt(Map<String,String> conf, String key, int def) {
+		String value = conf.get(key);
+		if (value == null) {
+			return def;
+		}
+		return Integer.parseInt(value);
+	}
+
+	//this derived from AvroOutputFormatBase.getCompressionCodec(..)
+	private CodecFactory getCompressionCodec(Map<String,String> conf) {
+		if (getBoolean(conf, CONF_COMPRESS, false)) {
+			int deflateLevel = getInt(conf, CONF_DEFLATE_LEVEL, CodecFactory.DEFAULT_DEFLATE_LEVEL);
+			int xzLevel = getInt(conf, CONF_XZ_LEVEL, CodecFactory.DEFAULT_XZ_LEVEL);
+
+			String outputCodec = conf.get(CONF_COMPRESS_CODEC);
+
+			if (DataFileConstants.DEFLATE_CODEC.equals(outputCodec)) {
+				return CodecFactory.deflateCodec(deflateLevel);
+			} else if (DataFileConstants.XZ_CODEC.equals(outputCodec)) {
+				return CodecFactory.xzCodec(xzLevel);
+			} else {
+				return CodecFactory.fromString(outputCodec);
+			}
+		}
+		return CodecFactory.nullCodec();
+	}
+
+	@Override
+	@SuppressWarnings("deprecation")
+	public void open(FileSystem fs, Path path) throws IOException {
+		super.open(fs, path);
+
+		CodecFactory compressionCodec = getCompressionCodec(properties);
+		Schema keySchema = Schema.parse(properties.get(CONF_OUTPUT_KEY_SCHEMA));
+		Schema valueSchema = Schema.parse(properties.get(CONF_OUTPUT_VALUE_SCHEMA));
+		keyValueWriter = new AvroKeyValueWriter<K, V>(keySchema, valueSchema, compressionCodec, getStream());
+	}
+
+	@Override
+	public void close() throws IOException {
+		super.close();//the order is important since super.close flushes inside
+		if (keyValueWriter != null) {
+			keyValueWriter.close();
+		}
+	}
+	
+	@Override
+	public long flush() throws IOException {
+		if (keyValueWriter != null) {
+			keyValueWriter.sync();
+		}
+		return super.flush();
+	}
+
+	@Override
+	public void write(Tuple2<K, V> element) throws IOException {
+		getStream(); // Throws if the stream is not open
+		keyValueWriter.write(element.f0, element.f1);
+	}
+
+	@Override
+	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
+		if (!type.isTupleType()) {
+			throw new IllegalArgumentException("Input TypeInformation is not a tuple type.");
+		}
+
+		TupleTypeInfoBase<?> tupleType = (TupleTypeInfoBase<?>) type;
+
+		if (tupleType.getArity() != 2) {
+			throw new IllegalArgumentException("Input TypeInformation must be a Tuple2 type.");
+		}
+	}
+
+	@Override
+	public Writer<Tuple2<K, V>> duplicate() {
+		return new AvroKeyValueSinkWriter<K, V>(properties);
+	}
+	
+	// taken from m/r avro lib to remove dependency on it
+	private static final class AvroKeyValueWriter<K, V> {
+		/** A writer for the Avro container file. */
+		private final DataFileWriter<GenericRecord> mAvroFileWriter;
+
+		/**
+		 * The writer schema for the generic record entries of the Avro
+		 * container file.
+		 */
+		private final Schema mKeyValuePairSchema;
+
+		/**
+		 * A reusable Avro generic record for writing key/value pairs to the
+		 * file.
+		 */
+		private final AvroKeyValue<Object, Object> mOutputRecord;
+
+		AvroKeyValueWriter(Schema keySchema, Schema valueSchema,
+				CodecFactory compressionCodec, OutputStream outputStream,
+				int syncInterval) throws IOException {
+			// Create the generic record schema for the key/value pair.
+			mKeyValuePairSchema = AvroKeyValue
+					.getSchema(keySchema, valueSchema);
+
+			// Create an Avro container file and a writer to it.
+			DatumWriter<GenericRecord> genericDatumWriter = new GenericDatumWriter<GenericRecord>(
+					mKeyValuePairSchema);
+			mAvroFileWriter = new DataFileWriter<GenericRecord>(
+					genericDatumWriter);
+			mAvroFileWriter.setCodec(compressionCodec);
+			mAvroFileWriter.setSyncInterval(syncInterval);
+			mAvroFileWriter.create(mKeyValuePairSchema, outputStream);
+
+			// Create a reusable output record.
+			mOutputRecord = new AvroKeyValue<Object, Object>(
+					new GenericData.Record(mKeyValuePairSchema));
+		}
+
+		AvroKeyValueWriter(Schema keySchema, Schema valueSchema,
+				CodecFactory compressionCodec, OutputStream outputStream)
+				throws IOException {
+			this(keySchema, valueSchema, compressionCodec, outputStream,
+					DataFileConstants.DEFAULT_SYNC_INTERVAL);
+		}
+
+		void write(K key, V value) throws IOException {
+			mOutputRecord.setKey(key);
+			mOutputRecord.setValue(value);
+			mAvroFileWriter.append(mOutputRecord.get());
+		}
+
+		void close() throws IOException {
+			mAvroFileWriter.close();
+		}
+
+		long sync() throws IOException {
+			return mAvroFileWriter.sync();
+		}
+	}
+
+	// taken from AvroKeyValue avro-mapr lib
+	public static class AvroKeyValue<K, V> {
+		/** The name of the key value pair generic record. */
+		public static final String KEY_VALUE_PAIR_RECORD_NAME = "KeyValuePair";
+
+		/** The namespace of the key value pair generic record. */
+		public static final String KEY_VALUE_PAIR_RECORD_NAMESPACE = "org.apache.avro.mapreduce";
+
+		/** The name of the generic record field containing the key. */
+		public static final String KEY_FIELD = "key";
+
+		/** The name of the generic record field containing the value. */
+		public static final String VALUE_FIELD = "value";
+
+		/** The key/value generic record wrapped by this class. */
+		public final GenericRecord mKeyValueRecord;
+
+		/**
+		 * Wraps a GenericRecord that is a key value pair.
+		 */
+		public AvroKeyValue(GenericRecord keyValueRecord) {
+			mKeyValueRecord = keyValueRecord;
+		}
+
+		public GenericRecord get() {
+			return mKeyValueRecord;
+		}
+
+		public void setKey(K key) {
+			mKeyValueRecord.put(KEY_FIELD, key);
+		}
+
+		public void setValue(V value) {
+			mKeyValueRecord.put(VALUE_FIELD, value);
+		}
+
+		@SuppressWarnings("unchecked")
+		public K getKey() {
+			return (K) mKeyValueRecord.get(KEY_FIELD);
+		}
+
+		@SuppressWarnings("unchecked")
+		public V getValue() {
+			return (V) mKeyValueRecord.get(VALUE_FIELD);
+		}
+
+		/**
+		 * Creates a KeyValuePair generic record schema.
+		 * 
+		 * @return A schema for a generic record with two fields: 'key' and
+		 *         'value'.
+		 */
+		public static Schema getSchema(Schema keySchema, Schema valueSchema) {
+			Schema schema = Schema.createRecord(KEY_VALUE_PAIR_RECORD_NAME,
+					"A key/value pair", KEY_VALUE_PAIR_RECORD_NAMESPACE, false);
+			schema.setFields(Arrays.asList(new Schema.Field(KEY_FIELD,
+					keySchema, "The key", null), new Schema.Field(VALUE_FIELD,
+					valueSchema, "The value", null)));
+			return schema;
+		}
+	}
+}

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

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

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

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


[16/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
deleted file mode 100644
index fc4a35e..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
+++ /dev/null
@@ -1,916 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs;
-
-import org.apache.commons.lang3.time.StopWatch;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
-import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink;
-import org.apache.flink.util.Preconditions;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-/**
- * Sink that emits its input elements to rolling {@link org.apache.hadoop.fs.FileSystem} files. This
- * is integrated with the checkpointing mechanism to provide exactly once semantics.
- *
- * <p>
- * When creating the sink a {@code basePath} must be specified. The base directory contains
- * one directory for every bucket. The bucket directories themselves contain several part files.
- * These contain the actual written data.
- *
- * <p>
- * The sink uses a {@link Bucketer} to determine the name of bucket directories inside the
- * base directory. Whenever the {@code Bucketer} returns a different directory name than
- * it returned before the sink will close the current part files inside that bucket
- * and start the new bucket directory. The default bucketer is a {@link DateTimeBucketer} with
- * date format string {@code ""yyyy-MM-dd--HH"}. You can specify a custom {@code Bucketer}
- * using {@link #setBucketer(Bucketer)}. For example, use
- * {@link NonRollingBucketer} if you don't want to have
- * buckets but still write part files in a fault-tolerant way.
- *
- * <p>
- * The filenames of the part files contain the part prefix, the parallel subtask index of the sink
- * and a rolling counter, for example {@code "part-1-17"}. Per default the part prefix is
- * {@code "part"} but this can be
- * configured using {@link #setPartPrefix(String)}. When a part file becomes bigger
- * than the batch size the current part file is closed, the part counter is increased and
- * a new part file is created. The batch size defaults to {@code 384MB}, this can be configured
- * using {@link #setBatchSize(long)}.
- *
- * <p>
- * Part files can be in one of three states: in-progress, pending or finished. The reason for this
- * is how the sink works together with the checkpointing mechanism to provide exactly-once semantics
- * and fault-tolerance. The part file that is currently being written to is in-progress. Once
- * a part file is closed for writing it becomes pending. When a checkpoint is successful the
- * currently pending files will be moved to finished. If a failure occurs the pending files
- * will be deleted to reset state to the last checkpoint. The data in in-progress files will
- * also have to be rolled back. If the {@code FileSystem} supports the {@code truncate} call
- * this will be used to reset the file back to a previous state. If not, a special file
- * with the same name as the part file and the suffix {@code ".valid-length"} will be written
- * that contains the length up to which the file contains valid data. When reading the file
- * it must be ensured that it is only read up to that point. The prefixes and suffixes for
- * the different file states and valid-length files can be configured, for example with
- * {@link #setPendingSuffix(String)}.
- *
- * <p>
- * Note: If checkpointing is not enabled the pending files will never be moved to the finished state.
- * In that case, the pending suffix/prefix can be set to {@code ""} to make the sink work
- * in a non-fault-tolerant way but still provide output without prefixes and suffixes.
- *
- * <p>
- * The part files are written using an instance of {@link Writer}. By default
- * {@link org.apache.flink.streaming.connectors.fs.StringWriter} is used, which writes the result
- * of {@code toString()} for every element. Separated by newlines. You can configure the writer
- * using {@link #setWriter(Writer)}. For example,
- * {@link org.apache.flink.streaming.connectors.fs.SequenceFileWriter} can be used to write
- * Hadoop {@code SequenceFiles}.
- *
- * <p>
- * Example:
- *
- * <pre>{@code
- *     new RollingSink<Tuple2<IntWritable, Text>>(outPath)
- *         .setWriter(new SequenceFileWriter<IntWritable, Text>())
- *         .setBucketer(new DateTimeBucketer("yyyy-MM-dd--HHmm")
- * }</pre>
- *
- * This will create a sink that writes to {@code SequenceFiles} and rolls every minute.
- *
- * @see DateTimeBucketer
- * @see StringWriter
- * @see SequenceFileWriter
- *
- * @param <T> Type of the elements emitted by this sink
- *
- * @deprecated use {@link BucketingSink} instead.
- */
-@Deprecated
-public class RollingSink<T> extends RichSinkFunction<T>
-		implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener {
-
-	private static final long serialVersionUID = 1L;
-
-	private static Logger LOG = LoggerFactory.getLogger(RollingSink.class);
-
-
-	// --------------------------------------------------------------------------------------------
-	//  User configuration values
-	// --------------------------------------------------------------------------------------------
-	// These are initialized with some defaults but are meant to be changeable by the user
-
-	/**
-	 * The default maximum size of part files (currently {@code 384 MB}).
-	 */
-	private final long DEFAULT_BATCH_SIZE = 1024L * 1024L * 384L;
-
-	/**
-	 * This is used for part files that we are writing to but which where not yet confirmed
-	 * by a checkpoint.
-	 */
-	private final String DEFAULT_IN_PROGRESS_SUFFIX = ".in-progress";
-
-	/**
-	 * See above, but for prefix
-	 */
-	private final String DEFAULT_IN_PROGRESS_PREFIX = "_";
-
-	/**
-	 * This is used for part files that we are not writing to but which are not yet confirmed by
-	 * checkpoint.
-	 */
-	private final String DEFAULT_PENDING_SUFFIX = ".pending";
-
-	/**
-	 * See above, but for prefix.
-	 */
-	private final String DEFAULT_PENDING_PREFIX = "_";
-
-	/**
-	 * When truncate() is not supported on the used FileSystem we instead write a
-	 * file along the part file with this ending that contains the length up to which
-	 * the part file is valid.
-	 */
-	private final String DEFAULT_VALID_SUFFIX = ".valid-length";
-
-	/**
-	 * See above, but for prefix.
-	 */
-	private final String DEFAULT_VALID_PREFIX = "_";
-
-	/**
-	 * The default prefix for part files.
-	 */
-	private final String DEFAULT_PART_REFIX = "part";
-
-	/**
-	 * The default timeout for asynchronous operations such as recoverLease and truncate. In
-	 * milliseconds.
-	 */
-	private final long DEFAULT_ASYNC_TIMEOUT_MS = 60 * 1000;
-
-
-	/**
-	 * The base {@code Path} that stores all bucket directories.
-	 */
-	private final String basePath;
-
-	/**
-	 * The {@code Bucketer} that is used to determine the path of bucket directories.
-	 */
-	private Bucketer bucketer;
-
-	/**
-	 * We have a template and call duplicate() for each parallel writer in open() to get the actual
-	 * writer that is used for the part files.
-	 */
-	private Writer<T> writerTemplate;
-
-	/**
-	 * The actual writer that we user for writing the part files.
-	 */
-	private Writer<T> writer;
-
-	/**
-	 * Maximum size of part files. If files exceed this we close and create a new one in the same
-	 * bucket directory.
-	 */
-	private long batchSize;
-
-	// These are the actually configured prefixes/suffixes
-	private String inProgressSuffix = DEFAULT_IN_PROGRESS_SUFFIX;
-	private String inProgressPrefix = DEFAULT_IN_PROGRESS_PREFIX;
-
-	private String pendingSuffix = DEFAULT_PENDING_SUFFIX;
-	private String pendingPrefix = DEFAULT_PENDING_PREFIX;
-
-	private String validLengthSuffix = DEFAULT_VALID_SUFFIX;
-	private String validLengthPrefix= DEFAULT_VALID_PREFIX;
-
-	private String partPrefix = DEFAULT_PART_REFIX;
-
-	/**
-	 * The timeout for asynchronous operations such as recoverLease and truncate. In
-	 * milliseconds.
-	 */
-	private long asyncTimeout = DEFAULT_ASYNC_TIMEOUT_MS;
-
-	// --------------------------------------------------------------------------------------------
-	//  Internal fields (not configurable by user)
-	// --------------------------------------------------------------------------------------------
-
-
-	/**
-	 * The part file that we are currently writing to.
-	 */
-	private transient Path currentPartPath;
-
-	/**
-	 * The bucket directory that we are currently filling.
-	 */
-	private transient Path currentBucketDirectory;
-
-	/**
-	 * For counting the part files inside a bucket directory. Part files follow the patter
-	 * {@code "{part-prefix}-{subtask}-{count}"}. When creating new part files we increase the counter.
-	 */
-	private transient int partCounter;
-
-	/**
-	 * Tracks if the writer is currently opened or closed.
-	 */
-	private transient boolean isWriterOpen;
-
-	/**
-	 * We use reflection to get the .truncate() method, this is only available starting with
-	 * Hadoop 2.7
-	 */
-	private transient Method refTruncate;
-
-	/**
-	 * The state object that is handled by flink from snapshot/restore. In there we store the
-	 * current part file path, the valid length of the in-progress files and pending part files.
-	 */
-	private transient BucketState bucketState;
-
-	private transient ListState<BucketState> restoredBucketStates;
-
-	/**
-	 * User-defined FileSystem parameters.
-     */
-	private Configuration fsConfig;
-
-	/**
-	 * The FileSystem reference.
-	 */
-	private transient FileSystem fs;
-	/**
-	 * Creates a new {@code RollingSink} that writes files to the given base directory.
-	 *
-	 * <p>
-	 * This uses a{@link DateTimeBucketer} as bucketer and a {@link StringWriter} has writer.
-	 * The maximum bucket size is set to 384 MB.
-	 *
-	 * @param basePath The directory to which to write the bucket files.
-	 */
-	public RollingSink(String basePath) {
-		this.basePath = basePath;
-		this.bucketer = new DateTimeBucketer();
-		this.batchSize = DEFAULT_BATCH_SIZE;
-		this.writerTemplate = new StringWriter<>();
-	}
-
-	/**
-	 * Specify a custom {@code Configuration} that will be used when creating
-	 * the {@link FileSystem} for writing.
-	 */
-	public RollingSink<T> setFSConfig(Configuration config) {
-		this.fsConfig = new Configuration();
-		fsConfig.addAll(config);
-		return this;
-	}
-
-	/**
-	 * Specify a custom {@code Configuration} that will be used when creating
-	 * the {@link FileSystem} for writing.
-	 */
-	public RollingSink<T> setFSConfig(org.apache.hadoop.conf.Configuration config) {
-		this.fsConfig = new Configuration();
-		for(Map.Entry<String, String> entry : config) {
-			fsConfig.setString(entry.getKey(), entry.getValue());
-		}
-		return this;
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
-		if (this.writerTemplate instanceof InputTypeConfigurable) {
-			((InputTypeConfigurable) writerTemplate).setInputType(type, executionConfig);
-		}
-	}
-
-	@Override
-	public void initializeState(FunctionInitializationContext context) throws Exception {
-		Preconditions.checkArgument(this.restoredBucketStates == null,
-			"The " + getClass().getSimpleName() + " has already been initialized.");
-
-		initFileSystem();
-
-		if (this.refTruncate == null) {
-			this.refTruncate = reflectTruncate(fs);
-		}
-
-		OperatorStateStore stateStore = context.getOperatorStateStore();
-		restoredBucketStates = stateStore.getSerializableListState("rolling-states");
-
-		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
-		if (context.isRestored()) {
-			LOG.info("Restoring state for the {} (taskIdx={}).", getClass().getSimpleName(), subtaskIndex);
-
-			for (BucketState bucketState : restoredBucketStates.get()) {
-				handleRestoredBucketState(bucketState);
-			}
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("{} (taskIdx= {}) restored {}", getClass().getSimpleName(), subtaskIndex, bucketState);
-			}
-		} else {
-			LOG.info("No state to restore for the {} (taskIdx= {}).", getClass().getSimpleName(), subtaskIndex);
-		}
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-
-		partCounter = 0;
-
-		this.writer = writerTemplate.duplicate();
-
-		bucketState = new BucketState();
-	}
-
-	/**
-	 * Create a file system with the user-defined hdfs config
-	 * @throws IOException
-	 */
-	private void initFileSystem() throws IOException {
-		if (fs != null) {
-			return;
-		}
-		org.apache.hadoop.conf.Configuration hadoopConf = HadoopFileSystem.getHadoopConfiguration();
-		if (fsConfig != null) {
-			String disableCacheName = String.format("fs.%s.impl.disable.cache", new Path(basePath).toUri().getScheme());
-			hadoopConf.setBoolean(disableCacheName, true);
-			for (String key : fsConfig.keySet()) {
-				hadoopConf.set(key, fsConfig.getString(key, null));
-			}
-		}
-
-		fs = new Path(basePath).getFileSystem(hadoopConf);
-	}
-
-	@Override
-	public void close() throws Exception {
-		closeCurrentPartFile();
-	}
-
-	@Override
-	public void invoke(T value) throws Exception {
-		if (shouldRoll()) {
-			openNewPartFile();
-		}
-		writer.write(value);
-	}
-
-	/**
-	 * Determines whether we should change the bucket file we are writing to.
-	 *
-	 * <p>
-	 * This will roll if no file was created yet, if the file size is larger than the specified size
-	 * or if the {@code Bucketer} determines that we should roll.
-	 */
-	private boolean shouldRoll() throws IOException {
-		boolean shouldRoll = false;
-		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
-		if (!isWriterOpen) {
-			shouldRoll = true;
-			LOG.debug("RollingSink {} starting new initial bucket. ", subtaskIndex);
-		}
-		if (bucketer.shouldStartNewBucket(new Path(basePath), currentBucketDirectory)) {
-			shouldRoll = true;
-			LOG.debug("RollingSink {} starting new bucket because {} said we should. ", subtaskIndex, bucketer);
-			// we will retrieve a new bucket base path in openNewPartFile so reset the part counter
-			partCounter = 0;
-		}
-		if (isWriterOpen) {
-			long writePosition = writer.getPos();
-			if (isWriterOpen && writePosition > batchSize) {
-				shouldRoll = true;
-				LOG.debug(
-						"RollingSink {} starting new bucket because file position {} is above batch size {}.",
-						subtaskIndex,
-						writePosition,
-						batchSize);
-			}
-		}
-		return shouldRoll;
-	}
-
-	/**
-	 * Opens a new part file.
-	 *
-	 * <p>
-	 * This closes the old bucket file and retrieves a new bucket path from the {@code Bucketer}.
-	 */
-	private void openNewPartFile() throws Exception {
-		closeCurrentPartFile();
-
-		Path newBucketDirectory = bucketer.getNextBucketPath(new Path(basePath));
-
-		if (!newBucketDirectory.equals(currentBucketDirectory)) {
-			currentBucketDirectory = newBucketDirectory;
-			try {
-				if (fs.mkdirs(currentBucketDirectory)) {
-					LOG.debug("Created new bucket directory: {}", currentBucketDirectory);
-				}
-			} catch (IOException e) {
-				throw new RuntimeException("Could not create base path for new rolling file.", e);
-			}
-		}
-
-		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
-		currentPartPath = new Path(currentBucketDirectory, partPrefix + "-" + subtaskIndex + "-" + partCounter);
-
-		// This should work since there is only one parallel subtask that tries names with
-		// our subtask id. Otherwise we would run into concurrency issues here.
-		while (fs.exists(currentPartPath) ||
-				fs.exists(getPendingPathFor(currentPartPath)) ||
-				fs.exists(getInProgressPathFor(currentPartPath))) {
-			partCounter++;
-			currentPartPath = new Path(currentBucketDirectory, partPrefix + "-" + subtaskIndex + "-" + partCounter);
-		}
-
-		// increase, so we don't have to check for this name next time
-		partCounter++;
-
-		LOG.debug("Next part path is {}", currentPartPath.toString());
-
-		Path inProgressPath = getInProgressPathFor(currentPartPath);
-		writer.open(fs, inProgressPath);
-		isWriterOpen = true;
-	}
-
-	private Path getPendingPathFor(Path path) {
-		return new Path(path.getParent(), pendingPrefix + path.getName()).suffix(pendingSuffix);
-	}
-
-	private Path getInProgressPathFor(Path path) {
-		return new Path(path.getParent(), inProgressPrefix + path.getName()).suffix(inProgressSuffix);
-	}
-
-	private Path getValidLengthPathFor(Path path) {
-		return new Path(path.getParent(), validLengthPrefix + path.getName()).suffix(validLengthSuffix);
-	}
-
-	/**
-	 * Closes the current part file.
-	 *
-	 * <p>
-	 * This moves the current in-progress part file to a pending file and adds it to the list
-	 * of pending files in our bucket state.
-	 */
-	private void closeCurrentPartFile() throws Exception {
-		if (isWriterOpen) {
-			writer.close();
-			isWriterOpen = false;
-		}
-
-		if (currentPartPath != null) {
-			Path inProgressPath = getInProgressPathFor(currentPartPath);
-			Path pendingPath = getPendingPathFor(currentPartPath);
-			fs.rename(inProgressPath, pendingPath);
-			LOG.debug("Moving in-progress bucket {} to pending file {}", inProgressPath, pendingPath);
-			this.bucketState.pendingFiles.add(currentPartPath.toString());
-		}
-	}
-
-	/**
-	 * Gets the truncate() call using reflection.
-	 * <p>
-	 * <b>NOTE: </b>This code comes from Flume
-	 */
-	private Method reflectTruncate(FileSystem fs) {
-		Method m = null;
-		if (fs != null) {
-			Class<?> fsClass = fs.getClass();
-			try {
-				m = fsClass.getMethod("truncate", Path.class, long.class);
-			} catch (NoSuchMethodException ex) {
-				LOG.debug("Truncate not found. Will write a file with suffix '{}' " +
-						" and prefix '{}' to specify how many bytes in a bucket are valid.", validLengthSuffix, validLengthPrefix);
-				return null;
-			}
-
-			// verify that truncate actually works
-			FSDataOutputStream outputStream;
-			Path testPath = new Path(UUID.randomUUID().toString());
-			try {
-				outputStream = fs.create(testPath);
-				outputStream.writeUTF("hello");
-				outputStream.close();
-			} catch (IOException e) {
-				LOG.error("Could not create file for checking if truncate works.", e);
-				throw new RuntimeException("Could not create file for checking if truncate works.", e);
-			}
-
-			try {
-				m.invoke(fs, testPath, 2);
-			} catch (IllegalAccessException | InvocationTargetException e) {
-				LOG.debug("Truncate is not supported.", e);
-				m = null;
-			}
-
-			try {
-				fs.delete(testPath, false);
-			} catch (IOException e) {
-				LOG.error("Could not delete truncate test file.", e);
-				throw new RuntimeException("Could not delete truncate test file.", e);
-			}
-		}
-		return m;
-	}
-
-	@Override
-	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		synchronized (bucketState.pendingFilesPerCheckpoint) {
-			Iterator<Map.Entry<Long, List<String>>> pendingCheckpointsIt =
-				bucketState.pendingFilesPerCheckpoint.entrySet().iterator();
-
-			while (pendingCheckpointsIt.hasNext()) {
-				Map.Entry<Long, List<String>> entry = pendingCheckpointsIt.next();
-				Long pastCheckpointId = entry.getKey();
-
-				if (pastCheckpointId <= checkpointId) {
-					LOG.debug("Moving pending files to final location for checkpoint {}", pastCheckpointId);
-					// All the pending files are buckets that have been completed but are waiting to be renamed
-					// to their final name
-					for (String filename : entry.getValue()) {
-						Path finalPath = new Path(filename);
-						Path pendingPath = getPendingPathFor(finalPath);
-
-						fs.rename(pendingPath, finalPath);
-						LOG.debug("Moving pending file {} to final location after complete checkpoint {}.",
-								pendingPath, pastCheckpointId);
-					}
-					pendingCheckpointsIt.remove();
-				}
-			}
-		}
-	}
-
-	@Override
-	public void snapshotState(FunctionSnapshotContext context) throws Exception {
-		Preconditions.checkNotNull(restoredBucketStates,
-			"The " + getClass().getSimpleName() + " has not been properly initialized.");
-
-		int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
-		
-		if (isWriterOpen) {
-			bucketState.currentFile = currentPartPath.toString();
-			bucketState.currentFileValidLength = writer.flush();
-		}
-
-		synchronized (bucketState.pendingFilesPerCheckpoint) {
-			bucketState.pendingFilesPerCheckpoint.put(context.getCheckpointId(), bucketState.pendingFiles);
-		}
-		bucketState.pendingFiles = new ArrayList<>();
-
-		restoredBucketStates.clear();
-		restoredBucketStates.add(bucketState);
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("{} (taskIdx={}) checkpointed {}.", getClass().getSimpleName(), subtaskIdx, bucketState);
-		}
-	}
-
-	private void handleRestoredBucketState(BucketState bucketState) {
-		// we can clean all the pending files since they were renamed to
-		// final files after this checkpoint was successful
-		// (we re-start from the last **successful** checkpoint)
-		bucketState.pendingFiles.clear();
-
-		if (bucketState.currentFile != null) {
-			// We were writing to a file when the last checkpoint occurred. This file can either
-			// be still in-progress or became a pending file at some point after the checkpoint.
-			// Either way, we have to truncate it back to a valid state (or write a .valid-length
-			// file that specifies up to which length it is valid) and rename it to the final name
-			// before starting a new bucket file.
-			Path partPath = new Path(bucketState.currentFile);
-			try {
-				Path partPendingPath = getPendingPathFor(partPath);
-				Path partInProgressPath = getInProgressPathFor(partPath);
-
-				if (fs.exists(partPendingPath)) {
-					LOG.debug("In-progress file {} has been moved to pending after checkpoint, moving to final location.", partPath);
-					// has been moved to pending in the mean time, rename to final location
-					fs.rename(partPendingPath, partPath);
-				} else if (fs.exists(partInProgressPath)) {
-					LOG.debug("In-progress file {} is still in-progress, moving to final location.", partPath);
-					// it was still in progress, rename to final path
-					fs.rename(partInProgressPath, partPath);
-				} else if (fs.exists(partPath)) {
-					LOG.debug("In-Progress file {} was already moved to final location {}.", bucketState.currentFile, partPath);
-				} else {
-					LOG.debug("In-Progress file {} was neither moved to pending nor is still in progress. Possibly, " +
-							"it was moved to final location by a previous snapshot restore", bucketState.currentFile);
-				}
-
-				if (this.refTruncate == null) {
-					this.refTruncate = reflectTruncate(fs);
-				}
-
-				// truncate it or write a ".valid-length" file to specify up to which point it is valid
-				if (refTruncate != null) {
-					LOG.debug("Truncating {} to valid length {}", partPath, bucketState.currentFileValidLength);
-					// some-one else might still hold the lease from a previous try, we are
-					// recovering, after all ...
-					if (fs instanceof DistributedFileSystem) {
-						DistributedFileSystem dfs = (DistributedFileSystem) fs;
-						LOG.debug("Trying to recover file lease {}", partPath);
-						dfs.recoverLease(partPath);
-						boolean isclosed= dfs.isFileClosed(partPath);
-						StopWatch sw = new StopWatch();
-						sw.start();
-						while(!isclosed) {
-							if(sw.getTime() > asyncTimeout) {
-								break;
-							}
-							try {
-								Thread.sleep(500);
-							} catch (InterruptedException e1) {
-								// ignore it
-							}
-							isclosed = dfs.isFileClosed(partPath);
-						}
-					}
-					Boolean truncated = (Boolean) refTruncate.invoke(fs, partPath, bucketState.currentFileValidLength);
-					if (!truncated) {
-						LOG.debug("Truncate did not immediately complete for {}, waiting...", partPath);
-
-						// we must wait for the asynchronous truncate operation to complete
-						StopWatch sw = new StopWatch();
-						sw.start();
-						long newLen = fs.getFileStatus(partPath).getLen();
-						while(newLen != bucketState.currentFileValidLength) {
-							if(sw.getTime() > asyncTimeout) {
-								break;
-							}
-							try {
-								Thread.sleep(500);
-							} catch (InterruptedException e1) {
-								// ignore it
-							}
-							newLen = fs.getFileStatus(partPath).getLen();
-						}
-						if (newLen != bucketState.currentFileValidLength) {
-							throw new RuntimeException("Truncate did not truncate to right length. Should be " + bucketState.currentFileValidLength + " is " + newLen + ".");
-						}
-					}
-
-				} else {
-					LOG.debug("Writing valid-length file for {} to specify valid length {}", partPath, bucketState.currentFileValidLength);
-					Path validLengthFilePath = getValidLengthPathFor(partPath);
-					if (!fs.exists(validLengthFilePath)) {
-						FSDataOutputStream lengthFileOut = fs.create(validLengthFilePath);
-						lengthFileOut.writeUTF(Long.toString(bucketState.currentFileValidLength));
-						lengthFileOut.close();
-					}
-				}
-
-				// invalidate in the state object
-				bucketState.currentFile = null;
-				bucketState.currentFileValidLength = -1;
-				isWriterOpen = false;
-			} catch (IOException e) {
-				LOG.error("Error while restoring RollingSink state.", e);
-				throw new RuntimeException("Error while restoring RollingSink state.", e);
-			} catch (InvocationTargetException | IllegalAccessException e) {
-				LOG.error("Could not invoke truncate.", e);
-				throw new RuntimeException("Could not invoke truncate.", e);
-			}
-		}
-
-		// Move files that are confirmed by a checkpoint but did not get moved to final location
-		// because the checkpoint notification did not happen before a failure
-
-		Set<Long> pastCheckpointIds = bucketState.pendingFilesPerCheckpoint.keySet();
-		LOG.debug("Moving pending files to final location on restore.");
-		for (Long pastCheckpointId : pastCheckpointIds) {
-			// All the pending files are buckets that have been completed but are waiting to be renamed
-			// to their final name
-			for (String filename : bucketState.pendingFilesPerCheckpoint.get(pastCheckpointId)) {
-				Path finalPath = new Path(filename);
-				Path pendingPath = getPendingPathFor(finalPath);
-
-				try {
-					if (fs.exists(pendingPath)) {
-						LOG.debug("(RESTORE) Moving pending file {} to final location after complete checkpoint {}.", pendingPath, pastCheckpointId);
-						fs.rename(pendingPath, finalPath);
-					}
-				} catch (IOException e) {
-					LOG.error("(RESTORE) Error while renaming pending file {} to final path {}: {}", pendingPath, finalPath, e);
-					throw new RuntimeException("Error while renaming pending file " + pendingPath+ " to final path " + finalPath, e);
-				}
-			}
-		}
-
-		synchronized (bucketState.pendingFilesPerCheckpoint) {
-			bucketState.pendingFilesPerCheckpoint.clear();
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Setters for User configuration values
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Sets the maximum bucket size in bytes.
-	 *
-	 * <p>
-	 * When a bucket part file becomes larger than this size a new bucket part file is started and
-	 * the old one is closed. The name of the bucket files depends on the {@link Bucketer}.
-	 *
-	 * @param batchSize The bucket part file size in bytes.
-	 */
-	public RollingSink<T> setBatchSize(long batchSize) {
-		this.batchSize = batchSize;
-		return this;
-	}
-
-	/**
-	 * Sets the {@link Bucketer} to use for determining the bucket files to write to.
-	 *
-	 * @param bucketer The bucketer to use.
-	 */
-	public RollingSink<T> setBucketer(Bucketer bucketer) {
-		this.bucketer = bucketer;
-		return this;
-	}
-
-	/**
-	 * Sets the {@link Writer} to be used for writing the incoming elements to bucket files.
-	 *
-	 * @param writer The {@code Writer} to use.
-	 */
-	public RollingSink<T> setWriter(Writer<T> writer) {
-		this.writerTemplate = writer;
-		return this;
-	}
-
-	/**
-	 * Sets the suffix of in-progress part files. The default is {@code "in-progress"}.
-	 */
-	public RollingSink<T> setInProgressSuffix(String inProgressSuffix) {
-		this.inProgressSuffix = inProgressSuffix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of in-progress part files. The default is {@code "_"}.
-	 */
-	public RollingSink<T> setInProgressPrefix(String inProgressPrefix) {
-		this.inProgressPrefix = inProgressPrefix;
-		return this;
-	}
-
-	/**
-	 * Sets the suffix of pending part files. The default is {@code ".pending"}.
-	 */
-	public RollingSink<T> setPendingSuffix(String pendingSuffix) {
-		this.pendingSuffix = pendingSuffix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of pending part files. The default is {@code "_"}.
-	 */
-	public RollingSink<T> setPendingPrefix(String pendingPrefix) {
-		this.pendingPrefix = pendingPrefix;
-		return this;
-	}
-
-	/**
-	 * Sets the suffix of valid-length files. The default is {@code ".valid-length"}.
-	 */
-	public RollingSink<T> setValidLengthSuffix(String validLengthSuffix) {
-		this.validLengthSuffix = validLengthSuffix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of valid-length files. The default is {@code "_"}.
-	 */
-	public RollingSink<T> setValidLengthPrefix(String validLengthPrefix) {
-		this.validLengthPrefix = validLengthPrefix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of part files.  The default is {@code "part"}.
-	 */
-	public RollingSink<T> setPartPrefix(String partPrefix) {
-		this.partPrefix = partPrefix;
-		return this;
-	}
-
-	/**
-	 * Disable cleanup of leftover in-progress/pending files when the sink is opened.
-	 *
-	 * <p>
-	 * This should only be disabled if using the sink without checkpoints, to not remove
-	 * the files already in the directory.
-	 *
-	 * @deprecated This option is deprecated and remains only for backwards compatibility.
-	 * We do not clean up lingering files anymore.
-	 */
-	@Deprecated
-	public RollingSink<T> disableCleanupOnOpen() {
-		return this;
-	}
-
-	/**
-	 * Sets the default timeout for asynchronous operations such as recoverLease and truncate.
-	 *
-	 * @param timeout The timeout, in milliseconds.
-	 */
-	public RollingSink<T> setAsyncTimeout(long timeout) {
-		this.asyncTimeout = timeout;
-		return this;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Internal Classes
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * This is used for keeping track of the current in-progress files and files that we mark
-	 * for moving from pending to final location after we get a checkpoint-complete notification.
-	 */
-	static final class BucketState implements Serializable {
-		private static final long serialVersionUID = 1L;
-
-		/**
-		 * The file that was in-progress when the last checkpoint occurred.
-		 */
-		String currentFile;
-
-		/**
-		 * The valid length of the in-progress file at the time of the last checkpoint.
-		 */
-		long currentFileValidLength = -1;
-
-		/**
-		 * Pending files that accumulated since the last checkpoint.
-		 */
-		List<String> pendingFiles = new ArrayList<>();
-
-		/**
-		 * When doing a checkpoint we move the pending files since the last checkpoint to this map
-		 * with the id of the checkpoint. When we get the checkpoint-complete notification we move
-		 * pending files of completed checkpoints to their final location.
-		 */
-		final Map<Long, List<String>> pendingFilesPerCheckpoint = new HashMap<>();
-
-		@Override
-		public String toString() {
-			return
-				"In-progress=" + currentFile +
-				" validLength=" + currentFileValidLength +
-				" pendingForNextCheckpoint=" + pendingFiles +
-				" pendingForPrevCheckpoints=" + pendingFilesPerCheckpoint;
-		}
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StreamWriterBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StreamWriterBase.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StreamWriterBase.java
deleted file mode 100644
index 140246f..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StreamWriterBase.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs;
-
-import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-
-/**
- * Base class for {@link Writer Writers} that write to a {@link FSDataOutputStream}.
- */
-public abstract class StreamWriterBase<T> implements Writer<T> {
-
-	private static Logger LOG = LoggerFactory.getLogger(BucketingSink.class);
-
-	/**
-	 * The {@code FSDataOutputStream} for the current part file.
-	 */
-	private transient FSDataOutputStream outStream;
-
-	/**
-	 * We use reflection to get the hflush method or use sync as a fallback.
-	 * The idea for this and the code comes from the Flume HDFS Sink.
-	 */
-	private transient Method refHflushOrSync;
-
-	/**
-	 * Returns the current output stream, if the stream is open.
-	 */
-	protected FSDataOutputStream getStream() {
-		if (outStream == null) {
-			throw new IllegalStateException("Output stream has not been opened");
-		}
-		return outStream;
-	}
-
-	/**
-	 * If hflush is available in this version of HDFS, then this method calls
-	 * hflush, else it calls sync.
-	 * @param os - The stream to flush/sync
-	 * @throws java.io.IOException
-	 *
-	 * <p>
-	 * Note: This code comes from Flume
-	 */
-	protected void hflushOrSync(FSDataOutputStream os) throws IOException {
-		try {
-			// At this point the refHflushOrSync cannot be null,
-			// since register method would have thrown if it was.
-			this.refHflushOrSync.invoke(os);
-		} catch (InvocationTargetException e) {
-			String msg = "Error while trying to hflushOrSync!";
-			LOG.error(msg + " " + e.getCause());
-			Throwable cause = e.getCause();
-			if(cause != null && cause instanceof IOException) {
-				throw (IOException)cause;
-			}
-			throw new RuntimeException(msg, e);
-		} catch (Exception e) {
-			String msg = "Error while trying to hflushOrSync!";
-			LOG.error(msg + " " + e);
-			throw new RuntimeException(msg, e);
-		}
-	}
-
-	/**
-	 * Gets the hflush call using reflection. Fallback to sync if hflush is not available.
-	 *
-	 * <p>
-	 * Note: This code comes from Flume
-	 */
-	private Method reflectHflushOrSync(FSDataOutputStream os) {
-		Method m = null;
-		if(os != null) {
-			Class<?> fsDataOutputStreamClass = os.getClass();
-			try {
-				m = fsDataOutputStreamClass.getMethod("hflush");
-			} catch (NoSuchMethodException ex) {
-				LOG.debug("HFlush not found. Will use sync() instead");
-				try {
-					m = fsDataOutputStreamClass.getMethod("sync");
-				} catch (Exception ex1) {
-					String msg = "Neither hflush not sync were found. That seems to be " +
-							"a problem!";
-					LOG.error(msg);
-					throw new RuntimeException(msg, ex1);
-				}
-			}
-		}
-		return m;
-	}
-
-	@Override
-	public void open(FileSystem fs, Path path) throws IOException {
-		if (outStream != null) {
-			throw new IllegalStateException("Writer has already been opened");
-		}
-		outStream = fs.create(path, false);
-		if (refHflushOrSync == null) {
-			refHflushOrSync = reflectHflushOrSync(outStream);
-		}
-	}
-
-	@Override
-	public long flush() throws IOException {
-		if (outStream == null) {
-			throw new IllegalStateException("Writer is not open");
-		}
-		hflushOrSync(outStream);
-		return outStream.getPos();
-	}
-
-	@Override
-	public long getPos() throws IOException {
-		if (outStream == null) {
-			throw new IllegalStateException("Writer is not open");
-		}
-		return outStream.getPos();
-	}
-
-	@Override
-	public void close() throws IOException {
-		if (outStream != null) {
-			flush();
-			outStream.close();
-			outStream = null;
-		}
-	}
-
-}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
deleted file mode 100644
index c3b4cb6..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs;
-
-import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * An implementation of {@code Writer} is used in conjunction with a
- * {@link BucketingSink} to perform the actual
- * writing to the bucket files.
- *
- * @param <T> The type of the elements that are being written by the sink.
- */
-public interface Writer<T> extends Serializable {
-
-	/**
-	 * Initializes the {@code Writer} for a newly opened bucket file.
-	 * Any internal per-bucket initialization should be performed here.
-	 *
-	 * @param fs The {@link org.apache.hadoop.fs.FileSystem} containing the newly opened file.
-	 * @param path The {@link org.apache.hadoop.fs.Path} of the newly opened file.
-	 */
-	void open(FileSystem fs, Path path) throws IOException;
-
-	/**
-	 * Flushes out any internally held data, and returns the offset that the file
-	 * must be truncated to at recovery.
-	 */
-	long flush() throws IOException;
-
-	/**
-	 * Retrieves the current position, and thus size, of the output file.
-	 */
-	long getPos() throws IOException;
-
-	/**
-	 * Closes the {@code Writer}. If the writer is already closed, no action will be
-	 * taken. The call should close all state related to the current output file,
-	 * including the output stream opened in {@code open}.
-	 */
-	void close() throws IOException ;
-
-	/**
-	 * Writes one element to the bucket file.
-	 */
-	void write(T element)throws IOException;
-
-	/**
-	 * Duplicates the {@code Writer}. This is used to get one {@code Writer} for each
-	 * parallel instance of the sink.
-	 */
-	Writer<T> duplicate();
-}

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

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


[24/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQConnectionConfig.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQConnectionConfig.java b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQConnectionConfig.java
new file mode 100644
index 0000000..72bac1c
--- /dev/null
+++ b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQConnectionConfig.java
@@ -0,0 +1,448 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.rabbitmq.common;
+
+import com.rabbitmq.client.ConnectionFactory;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.net.URISyntaxException;
+import java.security.KeyManagementException;
+import java.security.NoSuchAlgorithmException;
+
+/**
+ * Connection Configuration for RMQ.
+ * If {@link Builder#setUri(String)} has been set then {@link RMQConnectionConfig#RMQConnectionConfig(String, Integer,
+ * Boolean, Boolean, Integer, Integer, Integer, Integer)}
+ * will be used for initialize the RMQ connection or
+ * {@link RMQConnectionConfig#RMQConnectionConfig(String, Integer, String, String, String, Integer, Boolean,
+ * Boolean, Integer, Integer, Integer, Integer)}
+ * will be used for initialize the RMQ connection
+ */
+public class RMQConnectionConfig implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(RMQConnectionConfig.class);
+
+	private String host;
+	private Integer port;
+	private String virtualHost;
+	private String username;
+	private String password;
+	private String uri;
+
+	private Integer networkRecoveryInterval;
+	private Boolean automaticRecovery;
+	private Boolean topologyRecovery;
+
+	private Integer connectionTimeout;
+	private Integer requestedChannelMax;
+	private Integer requestedFrameMax;
+	private Integer requestedHeartbeat;
+
+	/**
+	*
+	* @param host host name
+	* @param port port
+	* @param virtualHost virtual host
+	* @param username username
+	* @param password password
+	* @param networkRecoveryInterval connection recovery interval in milliseconds
+	* @param automaticRecovery if automatic connection recovery
+	* @param topologyRecovery if topology recovery
+	* @param connectionTimeout connection timeout
+	* @param requestedChannelMax requested maximum channel number
+	* @param requestedFrameMax requested maximum frame size
+	* @param requestedHeartbeat requested heartbeat interval
+	* @throws NullPointerException if host or virtual host or username or password is null
+	*/
+	private RMQConnectionConfig(String host, Integer port, String virtualHost, String username, String password,
+								Integer networkRecoveryInterval, Boolean automaticRecovery,
+								Boolean topologyRecovery, Integer connectionTimeout, Integer requestedChannelMax,
+								Integer requestedFrameMax, Integer requestedHeartbeat){
+		Preconditions.checkNotNull(host, "host can not be null");
+		Preconditions.checkNotNull(port, "port can not be null");
+		Preconditions.checkNotNull(virtualHost, "virtualHost can not be null");
+		Preconditions.checkNotNull(username, "username can not be null");
+		Preconditions.checkNotNull(password, "password can not be null");
+		this.host = host;
+		this.port = port;
+		this.virtualHost = virtualHost;
+		this.username = username;
+		this.password = password;
+
+		this.networkRecoveryInterval = networkRecoveryInterval;
+		this.automaticRecovery = automaticRecovery;
+		this.topologyRecovery = topologyRecovery;
+		this.connectionTimeout = connectionTimeout;
+		this.requestedChannelMax = requestedChannelMax;
+		this.requestedFrameMax = requestedFrameMax;
+		this.requestedHeartbeat = requestedHeartbeat;
+	}
+
+	/**
+	*
+	* @param uri the connection URI
+	* @param networkRecoveryInterval connection recovery interval in milliseconds
+	* @param automaticRecovery if automatic connection recovery
+	* @param topologyRecovery if topology recovery
+	* @param connectionTimeout connection timeout
+	* @param requestedChannelMax requested maximum channel number
+	* @param requestedFrameMax requested maximum frame size
+	* @param requestedHeartbeat requested heartbeat interval
+	* @throws NullPointerException if URI is null
+	*/
+	private RMQConnectionConfig(String uri, Integer networkRecoveryInterval, Boolean automaticRecovery,
+								Boolean topologyRecovery, Integer connectionTimeout, Integer requestedChannelMax,
+								Integer requestedFrameMax, Integer requestedHeartbeat){
+		Preconditions.checkNotNull(uri, "Uri can not be null");
+		this.uri = uri;
+
+		this.networkRecoveryInterval = networkRecoveryInterval;
+		this.automaticRecovery = automaticRecovery;
+		this.topologyRecovery = topologyRecovery;
+		this.connectionTimeout = connectionTimeout;
+		this.requestedChannelMax = requestedChannelMax;
+		this.requestedFrameMax = requestedFrameMax;
+		this.requestedHeartbeat = requestedHeartbeat;
+	}
+
+	/** @return the host to use for connections */
+	public String getHost() {
+		return host;
+	}
+
+	/** @return the port to use for connections */
+	public int getPort() {
+		return port;
+	}
+
+	/**
+	 * Retrieve the virtual host.
+	 * @return the virtual host to use when connecting to the broker
+	 */
+	public String getVirtualHost() {
+		return virtualHost;
+	}
+
+	/**
+	 * Retrieve the user name.
+	 * @return the AMQP user name to use when connecting to the broker
+	 */
+	public String getUsername() {
+		return username;
+	}
+
+	/**
+	 * Retrieve the password.
+	 * @return the password to use when connecting to the broker
+	 */
+	public String getPassword() {
+		return password;
+	}
+
+	/**
+	 * Retrieve the URI.
+	 * @return the connection URI when connecting to the broker
+	 */
+	public String getUri() {
+		return uri;
+	}
+
+	/**
+	 * Returns automatic connection recovery interval in milliseconds.
+	 * @return how long will automatic recovery wait before attempting to reconnect, in ms; default is 5000
+	 */
+	public Integer getNetworkRecoveryInterval() {
+		return networkRecoveryInterval;
+	}
+
+	/**
+	 * Returns true if automatic connection recovery is enabled, false otherwise
+	 * @return true if automatic connection recovery is enabled, false otherwise
+	 */
+	public Boolean isAutomaticRecovery() {
+		return automaticRecovery;
+	}
+
+	/**
+	 * Returns true if topology recovery is enabled, false otherwise
+	 * @return true if topology recovery is enabled, false otherwise
+	 */
+	public Boolean isTopologyRecovery() {
+		return topologyRecovery;
+	}
+
+	/**
+	 * Retrieve the connection timeout.
+	 * @return the connection timeout, in milliseconds; zero for infinite
+	 */
+	public Integer getConnectionTimeout() {
+		return connectionTimeout;
+	}
+
+	/**
+	 * Retrieve the requested maximum channel number
+	 * @return the initially requested maximum channel number; zero for unlimited
+	 */
+	public Integer getRequestedChannelMax() {
+		return requestedChannelMax;
+	}
+
+	/**
+	 * Retrieve the requested maximum frame size
+	 * @return the initially requested maximum frame size, in octets; zero for unlimited
+	 */
+	public Integer getRequestedFrameMax() {
+		return requestedFrameMax;
+	}
+
+	/**
+	 * Retrieve the requested heartbeat interval.
+	 * @return the initially requested heartbeat interval, in seconds; zero for none
+	 */
+	public Integer getRequestedHeartbeat() {
+		return requestedHeartbeat;
+	}
+
+	/**
+	 *
+	 * @return Connection Factory for RMQ
+	 * @throws URISyntaxException, NoSuchAlgorithmException, KeyManagementException if Malformed URI has been passed
+	 */
+	public ConnectionFactory getConnectionFactory() throws URISyntaxException,
+		NoSuchAlgorithmException, KeyManagementException {
+		ConnectionFactory factory = new ConnectionFactory();
+		if (this.uri != null && !this.uri.isEmpty()){
+			try {
+				factory.setUri(this.uri);
+			} catch (URISyntaxException | NoSuchAlgorithmException | KeyManagementException e) {
+				LOG.error("Failed to parse uri", e);
+				throw e;
+			}
+		} else {
+			factory.setHost(this.host);
+			factory.setPort(this.port);
+			factory.setVirtualHost(this.virtualHost);
+			factory.setUsername(this.username);
+			factory.setPassword(this.password);
+		}
+
+		if (this.automaticRecovery != null) {
+			factory.setAutomaticRecoveryEnabled(this.automaticRecovery);
+		}
+		if (this.connectionTimeout != null) {
+			factory.setConnectionTimeout(this.connectionTimeout);
+		}
+		if (this.networkRecoveryInterval != null) {
+			factory.setNetworkRecoveryInterval(this.networkRecoveryInterval);
+		}
+		if (this.requestedHeartbeat != null) {
+			factory.setRequestedHeartbeat(this.requestedHeartbeat);
+		}
+		if (this.topologyRecovery != null) {
+			factory.setTopologyRecoveryEnabled(this.topologyRecovery);
+		}
+		if (this.requestedChannelMax != null) {
+			factory.setRequestedChannelMax(this.requestedChannelMax);
+		}
+		if (this.requestedFrameMax != null) {
+			factory.setRequestedFrameMax(this.requestedFrameMax);
+		}
+
+		return factory;
+	}
+
+	/**
+	 * The Builder Class for {@link RMQConnectionConfig}
+	 */
+	public static class Builder {
+
+		private String host;
+		private Integer port;
+		private String virtualHost;
+		private String username;
+		private String password;
+
+		private Integer networkRecoveryInterval;
+		private Boolean automaticRecovery;
+		private Boolean topologyRecovery;
+
+		private Integer connectionTimeout;
+		private Integer requestedChannelMax;
+		private Integer requestedFrameMax;
+		private Integer requestedHeartbeat;
+
+		private String uri;
+
+		/**
+		 * Set the target port.
+		 * @param port the default port to use for connections
+		 * @return the Builder
+		 */
+		public Builder setPort(int port) {
+			this.port = port;
+			return this;
+		}
+
+		/** @param host the default host to use for connections
+		 * @return the Builder
+		 */
+		public Builder setHost(String host) {
+			this.host = host;
+			return this;
+		}
+
+		/**
+		 * Set the virtual host.
+		 * @param virtualHost the virtual host to use when connecting to the broker
+		 * @return the Builder
+		 */
+		public Builder setVirtualHost(String virtualHost) {
+			this.virtualHost = virtualHost;
+			return this;
+		}
+
+		/**
+		 * Set the user name.
+		 * @param username the AMQP user name to use when connecting to the broker
+		 * @return the Builder
+		 */
+		public Builder setUserName(String username) {
+			this.username = username;
+			return this;
+		}
+
+		/**
+		 * Set the password.
+		 * @param password the password to use when connecting to the broker
+		 * @return the Builder
+		 */
+		public Builder setPassword(String password) {
+			this.password = password;
+			return this;
+		}
+
+		/**
+		 * Convenience method for setting the fields in an AMQP URI: host,
+		 * port, username, password and virtual host.  If any part of the
+		 * URI is ommited, the ConnectionFactory's corresponding variable
+		 * is left unchanged.
+		 * @param uri is the AMQP URI containing the data
+		 * @return the Builder
+		 */
+		public Builder setUri(String uri) {
+			this.uri = uri;
+			return this;
+		}
+
+		/**
+		 * Enables or disables topology recovery
+		 * @param topologyRecovery if true, enables topology recovery
+		 * @return the Builder
+		 */
+		public Builder setTopologyRecoveryEnabled(boolean topologyRecovery) {
+			this.topologyRecovery = topologyRecovery;
+			return this;
+		}
+
+		/**
+		 * Set the requested heartbeat.
+		 * @param requestedHeartbeat the initially requested heartbeat interval, in seconds; zero for none
+		 * @return the Builder
+		 */
+		public Builder setRequestedHeartbeat(int requestedHeartbeat) {
+			this.requestedHeartbeat = requestedHeartbeat;
+			return this;
+		}
+
+		/**
+		 * Set the requested maximum frame size
+		 * @param requestedFrameMax initially requested maximum frame size, in octets; zero for unlimited
+		 * @return the Builder
+		 */
+		public Builder setRequestedFrameMax(int requestedFrameMax) {
+			this.requestedFrameMax = requestedFrameMax;
+			return this;
+		}
+
+		/**
+		 * Set the requested maximum channel number
+		 * @param requestedChannelMax initially requested maximum channel number; zero for unlimited
+		 */
+		public Builder setRequestedChannelMax(int requestedChannelMax) {
+			this.requestedChannelMax = requestedChannelMax;
+			return this;
+		}
+
+		/**
+		 * Sets connection recovery interval. Default is 5000.
+		 * @param networkRecoveryInterval how long will automatic recovery wait before attempting to reconnect, in ms
+		 * @return the Builder
+		 */
+		public Builder setNetworkRecoveryInterval(int networkRecoveryInterval) {
+			this.networkRecoveryInterval = networkRecoveryInterval;
+			return this;
+		}
+
+		/**
+		 * Set the connection timeout.
+		 * @param connectionTimeout connection establishment timeout in milliseconds; zero for infinite
+		 * @return the Builder
+		 */
+		public Builder setConnectionTimeout(int connectionTimeout) {
+			this.connectionTimeout = connectionTimeout;
+			return this;
+		}
+
+		/**
+		 * Enables or disables automatic connection recovery
+		 * @param automaticRecovery if true, enables connection recovery
+		 * @return the Builder
+		 */
+		public Builder setAutomaticRecovery(boolean automaticRecovery) {
+			this.automaticRecovery = automaticRecovery;
+			return this;
+		}
+
+		/**
+		 * The Builder method
+		 * If URI is NULL we use host, port, vHost, username, password combination
+		 * to initialize connection. using  {@link RMQConnectionConfig#RMQConnectionConfig(String, Integer, String, String, String,
+		 * Integer, Boolean, Boolean, Integer, Integer, Integer, Integer)}
+		 *
+		 * else URI will be used to initialize the client connection
+		 * {@link RMQConnectionConfig#RMQConnectionConfig(String, Integer, Boolean, Boolean, Integer, Integer, Integer, Integer)}
+		 * @return RMQConnectionConfig
+		 */
+		public RMQConnectionConfig build(){
+			if(this.uri != null) {
+				return new RMQConnectionConfig(this.uri, this.networkRecoveryInterval,
+					this.automaticRecovery, this.topologyRecovery, this.connectionTimeout, this.requestedChannelMax,
+					this.requestedFrameMax, this.requestedHeartbeat);
+			} else {
+				return new RMQConnectionConfig(this.host, this.port, this.virtualHost, this.username, this.password,
+					this.networkRecoveryInterval, this.automaticRecovery, this.topologyRecovery,
+					this.connectionTimeout, this.requestedChannelMax, this.requestedFrameMax, this.requestedHeartbeat);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
new file mode 100644
index 0000000..b63c835
--- /dev/null
+++ b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
@@ -0,0 +1,419 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.rabbitmq;
+
+import com.rabbitmq.client.AMQP;
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.ConnectionFactory;
+import com.rabbitmq.client.Envelope;
+import com.rabbitmq.client.QueueingConsumer;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.SerializedCheckpointData;
+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.rabbitmq.common.RMQConnectionConfig;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.powermock.modules.junit4.PowerMockRunner;
+import com.rabbitmq.client.Connection;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+
+/**
+ * Tests for the RMQSource. The source supports two operation modes.
+ * 1) Exactly-once (when checkpointed) with RabbitMQ transactions and the deduplication mechanism in
+ *    {@link org.apache.flink.streaming.api.functions.source.MessageAcknowledgingSourceBase}.
+ * 2) At-least-once (when checkpointed) with RabbitMQ transactions but not deduplication.
+ * 3) No strong delivery guarantees (without checkpointing) with RabbitMQ auto-commit mode.
+ *
+ * This tests assumes that the message ids are increasing monotonously. That doesn't have to be the
+ * case. The correlation id is used to uniquely identify messages.
+ */
+@RunWith(PowerMockRunner.class)
+public class RMQSourceTest {
+
+	private RMQSource<String> source;
+
+	private Configuration config = new Configuration();
+
+	private Thread sourceThread;
+
+	private volatile long messageId;
+
+	private boolean generateCorrelationIds;
+
+	private volatile Exception exception;
+
+	@Before
+	public void beforeTest() throws Exception {
+
+		source = new RMQTestSource();
+		source.open(config);
+
+		messageId = 0;
+		generateCorrelationIds = true;
+
+		sourceThread = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					source.run(new DummySourceContext());
+				} catch (Exception e) {
+					exception = e;
+				}
+			}
+		});
+	}
+
+	@After
+	public void afterTest() throws Exception {
+		source.cancel();
+		sourceThread.join();
+	}
+
+	@Test
+	public void throwExceptionIfConnectionFactoryReturnNull() throws Exception {
+		RMQConnectionConfig connectionConfig = Mockito.mock(RMQConnectionConfig.class);
+		ConnectionFactory connectionFactory = Mockito.mock(ConnectionFactory.class);
+		Connection connection = Mockito.mock(Connection.class);
+		Mockito.when(connectionConfig.getConnectionFactory()).thenReturn(connectionFactory);
+		Mockito.when(connectionFactory.newConnection()).thenReturn(connection);
+		Mockito.when(connection.createChannel()).thenReturn(null);
+
+		RMQSource<String> rmqSource = new RMQSource<>(
+			connectionConfig, "queueDummy", true, new StringDeserializationScheme());
+		try {
+			rmqSource.open(new Configuration());
+		} catch (RuntimeException ex) {
+			assertEquals("None of RabbitMQ channels are available", ex.getMessage());
+		}
+	}
+
+	@Test
+	public void testCheckpointing() throws Exception {
+		source.autoAck = false;
+		sourceThread.start();
+
+		Thread.sleep(5);
+
+		final Random random = new Random(System.currentTimeMillis());
+		int numSnapshots = 50;
+		long previousSnapshotId;
+		long lastSnapshotId = 0;
+
+		long totalNumberOfAcks = 0;
+
+		for (int i=0; i < numSnapshots; i++) {
+			long snapshotId = random.nextLong();
+			SerializedCheckpointData[] data;
+
+			synchronized (DummySourceContext.lock) {
+				data = source.snapshotState(snapshotId, System.currentTimeMillis());
+				previousSnapshotId = lastSnapshotId;
+				lastSnapshotId = messageId;
+			}
+			// let some time pass
+			Thread.sleep(5);
+
+			// check if the correct number of messages have been snapshotted
+			final long numIds = lastSnapshotId - previousSnapshotId;
+			assertEquals(numIds, data[0].getNumIds());
+			// deserialize and check if the last id equals the last snapshotted id
+			ArrayDeque<Tuple2<Long, List<String>>> deque = SerializedCheckpointData.toDeque(data, new StringSerializer());
+			List<String> messageIds = deque.getLast().f1;
+			if (messageIds.size() > 0) {
+				assertEquals(lastSnapshotId, (long) Long.valueOf(messageIds.get(messageIds.size() - 1)));
+			}
+
+			// check if the messages are being acknowledged and the transaction comitted
+			synchronized (DummySourceContext.lock) {
+				source.notifyCheckpointComplete(snapshotId);
+			}
+			totalNumberOfAcks += numIds;
+
+		}
+
+		Mockito.verify(source.channel, Mockito.times((int) totalNumberOfAcks)).basicAck(Mockito.anyLong(), Mockito.eq(false));
+		Mockito.verify(source.channel, Mockito.times(numSnapshots)).txCommit();
+
+	}
+
+	/**
+	 * Checks whether recurring ids are processed again (they shouldn't be).
+	 */
+	@Test
+	public void testDuplicateId() throws Exception {
+		source.autoAck = false;
+		sourceThread.start();
+
+		while (messageId < 10) {
+			// wait until messages have been processed
+			Thread.sleep(5);
+		}
+
+		long oldMessageId;
+		synchronized (DummySourceContext.lock) {
+			oldMessageId = messageId;
+			messageId = 0;
+		}
+
+		while (messageId < 10) {
+			// process again
+			Thread.sleep(5);
+		}
+
+		synchronized (DummySourceContext.lock) {
+			assertEquals(Math.max(messageId, oldMessageId), DummySourceContext.numElementsCollected);
+		}
+	}
+
+
+	/**
+	 * The source should not acknowledge ids in auto-commit mode or check for previously acknowledged ids
+	 */
+	@Test
+	public void testCheckpointingDisabled() throws Exception {
+		source.autoAck = true;
+		sourceThread.start();
+
+		while (DummySourceContext.numElementsCollected < 50) {
+			// wait until messages have been processed
+			Thread.sleep(5);
+		}
+
+		// see addId in RMQTestSource.addId for the assert
+	}
+
+	/**
+	 * Tests error reporting in case of invalid correlation ids
+	 */
+	@Test
+	public void testCorrelationIdNotSet() throws InterruptedException {
+		generateCorrelationIds = false;
+		source.autoAck = false;
+		sourceThread.start();
+
+		sourceThread.join();
+
+		assertNotNull(exception);
+		assertTrue(exception instanceof NullPointerException);
+	}
+
+	/**
+	 * Tests whether constructor params are passed correctly.
+	 */
+	@Test
+	public void testConstructorParams() throws Exception {
+		// verify construction params
+		RMQConnectionConfig.Builder builder = new RMQConnectionConfig.Builder();
+		builder.setHost("hostTest").setPort(999).setUserName("userTest").setPassword("passTest").setVirtualHost("/");
+		ConstructorTestClass testObj = new ConstructorTestClass(
+			builder.build(), "queueTest", false, new StringDeserializationScheme());
+
+		try {
+			testObj.open(new Configuration());
+		} catch (Exception e) {
+			// connection fails but check if args have been passed correctly
+		}
+
+		assertEquals("hostTest", testObj.getFactory().getHost());
+		assertEquals(999, testObj.getFactory().getPort());
+		assertEquals("userTest", testObj.getFactory().getUsername());
+		assertEquals("passTest", testObj.getFactory().getPassword());
+	}
+
+	private static class ConstructorTestClass extends RMQSource<String> {
+
+		private ConnectionFactory factory;
+
+		public ConstructorTestClass(RMQConnectionConfig rmqConnectionConfig,
+									String queueName,
+									boolean usesCorrelationId,
+									DeserializationSchema<String> deserializationSchema) throws Exception {
+			super(rmqConnectionConfig, queueName, usesCorrelationId, deserializationSchema);
+			RMQConnectionConfig.Builder builder = new RMQConnectionConfig.Builder();
+			builder.setHost("hostTest").setPort(999).setUserName("userTest").setPassword("passTest").setVirtualHost("/");
+			factory = Mockito.spy(builder.build().getConnectionFactory());
+			try {
+				Mockito.doThrow(new RuntimeException()).when(factory).newConnection();
+			} catch (IOException e) {
+				fail("Failed to stub connection method");
+			}
+		}
+
+		@Override
+		protected ConnectionFactory setupConnectionFactory() {
+			return factory;
+		}
+
+		public ConnectionFactory getFactory() {
+			return factory;
+		}
+	}
+
+	private static class StringDeserializationScheme implements DeserializationSchema<String> {
+
+		@Override
+		public String deserialize(byte[] message) throws IOException {
+			try {
+				// wait a bit to not cause too much cpu load
+				Thread.sleep(1);
+			} catch (InterruptedException e) {
+				e.printStackTrace();
+			}
+			return new String(message);
+		}
+
+		@Override
+		public boolean isEndOfStream(String nextElement) {
+			return false;
+		}
+
+		@Override
+		public TypeInformation<String> getProducedType() {
+			return TypeExtractor.getForClass(String.class);
+		}
+	}
+
+	private class RMQTestSource extends RMQSource<String> {
+
+		public RMQTestSource() {
+			super(new RMQConnectionConfig.Builder().setHost("hostTest")
+					.setPort(999).setUserName("userTest").setPassword("passTest").setVirtualHost("/").build()
+				, "queueDummy", true, new StringDeserializationScheme());
+		}
+
+		@Override
+		public void open(Configuration config) throws Exception {
+			super.open(config);
+
+			consumer = Mockito.mock(QueueingConsumer.class);
+
+			// Mock for delivery
+			final QueueingConsumer.Delivery deliveryMock = Mockito.mock(QueueingConsumer.Delivery.class);
+			Mockito.when(deliveryMock.getBody()).thenReturn("test".getBytes());
+
+			try {
+				Mockito.when(consumer.nextDelivery()).thenReturn(deliveryMock);
+			} catch (InterruptedException e) {
+				fail("Couldn't setup up deliveryMock");
+			}
+
+			// Mock for envelope
+			Envelope envelope = Mockito.mock(Envelope.class);
+			Mockito.when(deliveryMock.getEnvelope()).thenReturn(envelope);
+
+			Mockito.when(envelope.getDeliveryTag()).thenAnswer(new Answer<Long>() {
+				@Override
+				public Long answer(InvocationOnMock invocation) throws Throwable {
+					return ++messageId;
+				}
+			});
+
+			// Mock for properties
+			AMQP.BasicProperties props = Mockito.mock(AMQP.BasicProperties.class);
+			Mockito.when(deliveryMock.getProperties()).thenReturn(props);
+
+			Mockito.when(props.getCorrelationId()).thenAnswer(new Answer<String>() {
+				@Override
+				public String answer(InvocationOnMock invocation) throws Throwable {
+					return generateCorrelationIds ? "" + messageId : null;
+				}
+			});
+
+		}
+
+		@Override
+		protected ConnectionFactory setupConnectionFactory() {
+			ConnectionFactory connectionFactory = Mockito.mock(ConnectionFactory.class);
+			Connection connection = Mockito.mock(Connection.class);
+			try {
+				Mockito.when(connectionFactory.newConnection()).thenReturn(connection);
+				Mockito.when(connection.createChannel()).thenReturn(Mockito.mock(Channel.class));
+			} catch (IOException e) {
+				fail("Test environment couldn't be created.");
+			}
+			return connectionFactory;
+		}
+
+		@Override
+		public RuntimeContext getRuntimeContext() {
+			return Mockito.mock(StreamingRuntimeContext.class);
+		}
+
+		@Override
+		protected boolean addId(String uid) {
+			assertEquals(false, autoAck);
+			return super.addId(uid);
+		}
+	}
+
+	private static class DummySourceContext implements SourceFunction.SourceContext<String> {
+
+		private static final Object lock = new Object();
+
+		private static long numElementsCollected;
+
+		public DummySourceContext() {
+			numElementsCollected = 0;
+		}
+
+		@Override
+		public void collect(String element) {
+			numElementsCollected++;
+		}
+
+		@Override
+		public void collectWithTimestamp(java.lang.String element, long timestamp) {
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return lock;
+		}
+
+		@Override
+		public void close() {
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQConnectionConfigTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQConnectionConfigTest.java b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQConnectionConfigTest.java
new file mode 100644
index 0000000..40985ce
--- /dev/null
+++ b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQConnectionConfigTest.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.rabbitmq.common;
+
+import com.rabbitmq.client.ConnectionFactory;
+import org.junit.Test;
+
+import java.net.URISyntaxException;
+import java.security.KeyManagementException;
+import java.security.NoSuchAlgorithmException;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class RMQConnectionConfigTest {
+
+	@Test(expected = NullPointerException.class)
+	public void shouldThrowNullPointExceptionIfHostIsNull() throws NoSuchAlgorithmException,
+		KeyManagementException, URISyntaxException {
+		RMQConnectionConfig connectionConfig = new RMQConnectionConfig.Builder()
+			.setPort(1000).setUserName("guest")
+			.setPassword("guest").setVirtualHost("/").build();
+		connectionConfig.getConnectionFactory();
+	}
+	@Test(expected = NullPointerException.class)
+	public void shouldThrowNullPointExceptionIfPortIsNull() throws NoSuchAlgorithmException,
+		KeyManagementException, URISyntaxException {
+		RMQConnectionConfig connectionConfig = new RMQConnectionConfig.Builder()
+			.setHost("localhost").setUserName("guest")
+			.setPassword("guest").setVirtualHost("/").build();
+		connectionConfig.getConnectionFactory();
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void shouldSetDefaultValueIfConnectionTimeoutNotGiven() throws NoSuchAlgorithmException,
+		KeyManagementException, URISyntaxException {
+		RMQConnectionConfig connectionConfig = new RMQConnectionConfig.Builder()
+			.setHost("localhost").setUserName("guest")
+			.setPassword("guest").setVirtualHost("/").build();
+		ConnectionFactory factory = connectionConfig.getConnectionFactory();
+		assertEquals(ConnectionFactory.DEFAULT_CONNECTION_TIMEOUT, factory.getConnectionTimeout());
+	}
+
+	@Test
+	public void shouldSetProvidedValueIfConnectionTimeoutNotGiven() throws NoSuchAlgorithmException,
+		KeyManagementException, URISyntaxException {
+		RMQConnectionConfig connectionConfig = new RMQConnectionConfig.Builder()
+			.setHost("localhost").setPort(5000).setUserName("guest")
+			.setPassword("guest").setVirtualHost("/")
+			.setConnectionTimeout(5000).build();
+		ConnectionFactory factory = connectionConfig.getConnectionFactory();
+		assertEquals(5000, factory.getConnectionTimeout());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQSinkTest.java b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQSinkTest.java
new file mode 100644
index 0000000..199cd1e
--- /dev/null
+++ b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/common/RMQSinkTest.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.rabbitmq.common;
+
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.connectors.rabbitmq.RMQSink;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
+
+public class RMQSinkTest {
+
+	private static final String QUEUE_NAME = "queue";
+	private static final String MESSAGE_STR = "msg";
+	private static final byte[] MESSAGE = new byte[1];
+
+	private RMQConnectionConfig rmqConnectionConfig;
+	private ConnectionFactory connectionFactory;
+	private Connection connection;
+	private Channel channel;
+	private SerializationSchema<String> serializationSchema;
+
+
+	@Before
+	public void before() throws Exception {
+		serializationSchema = spy(new DummySerializationSchema());
+		rmqConnectionConfig = mock(RMQConnectionConfig.class);
+		connectionFactory = mock(ConnectionFactory.class);
+		connection = mock(Connection.class);
+		channel = mock(Channel.class);
+
+		when(rmqConnectionConfig.getConnectionFactory()).thenReturn(connectionFactory);
+		when(connectionFactory.newConnection()).thenReturn(connection);
+		when(connection.createChannel()).thenReturn(channel);
+	}
+
+	@Test
+	public void openCallDeclaresQueue() throws Exception {
+		createRMQSink();
+
+		verify(channel).queueDeclare(QUEUE_NAME, false, false, false, null);
+	}
+
+	@Test
+	public void throwExceptionIfChannelIsNull() throws Exception {
+		when(connection.createChannel()).thenReturn(null);
+		try {
+			createRMQSink();
+		} catch (RuntimeException ex) {
+			assertEquals("None of RabbitMQ channels are available", ex.getMessage());
+		}
+	}
+
+	private RMQSink<String> createRMQSink() throws Exception {
+		RMQSink rmqSink = new RMQSink<String>(rmqConnectionConfig, QUEUE_NAME, serializationSchema);
+		rmqSink.open(new Configuration());
+		return rmqSink;
+	}
+
+	@Test
+	public void invokePublishBytesToQueue() throws Exception {
+		RMQSink<String> rmqSink = createRMQSink();
+
+		rmqSink.invoke(MESSAGE_STR);
+		verify(serializationSchema).serialize(MESSAGE_STR);
+		verify(channel).basicPublish("", QUEUE_NAME, null, MESSAGE);
+	}
+
+	@Test(expected = RuntimeException.class)
+	public void exceptionDuringPublishingIsNotIgnored() throws Exception {
+		RMQSink<String> rmqSink = createRMQSink();
+
+		doThrow(IOException.class).when(channel).basicPublish("", QUEUE_NAME, null, MESSAGE);
+		rmqSink.invoke("msg");
+	}
+
+	@Test
+	public void exceptionDuringPublishingIsIgnoredIfLogFailuresOnly() throws Exception {
+		RMQSink<String> rmqSink = createRMQSink();
+		rmqSink.setLogFailuresOnly(true);
+
+		doThrow(IOException.class).when(channel).basicPublish("", QUEUE_NAME, null, MESSAGE);
+		rmqSink.invoke("msg");
+	}
+
+	@Test
+	public void closeAllResources() throws Exception {
+		RMQSink<String> rmqSink = createRMQSink();
+
+		rmqSink.close();
+
+		verify(channel).close();
+		verify(connection).close();
+	}
+
+	private class DummySerializationSchema implements SerializationSchema<String> {
+		@Override
+		public byte[] serialize(String element) {
+			return MESSAGE;
+		}
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisSink.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisSink.java
new file mode 100644
index 0000000..f6b0fd7
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisSink.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.redis;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisClusterConfig;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisConfigBase;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisPoolConfig;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisSentinelConfig;
+import org.apache.flink.streaming.connectors.redis.common.container.RedisCommandsContainer;
+import org.apache.flink.streaming.connectors.redis.common.container.RedisCommandsContainerBuilder;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisDataType;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommandDescription;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisMapper;
+
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * A sink that delivers data to a Redis channel using the Jedis client.
+ * <p> The sink takes two arguments {@link FlinkJedisConfigBase} and {@link RedisMapper}.
+ * <p> When {@link FlinkJedisPoolConfig} is passed as the first argument,
+ * the sink will create connection using {@link redis.clients.jedis.JedisPool}. Please use this when
+ * you want to connect to a single Redis server.
+ * <p> When {@link FlinkJedisSentinelConfig} is passed as the first argument, the sink will create connection
+ * using {@link redis.clients.jedis.JedisSentinelPool}. Please use this when you want to connect to Sentinel.
+ * <p> Please use {@link FlinkJedisClusterConfig} as the first argument if you want to connect to
+ * a Redis Cluster.
+ *
+ * <p>Example:
+ *
+ * <pre>
+ *{@code
+ *public static class RedisExampleMapper implements RedisMapper<Tuple2<String, String>> {
+ *
+ *	private RedisCommand redisCommand;
+ *
+ *	public RedisExampleMapper(RedisCommand redisCommand){
+ *		this.redisCommand = redisCommand;
+ *	}
+ *	public RedisCommandDescription getCommandDescription() {
+ *		return new RedisCommandDescription(redisCommand, REDIS_ADDITIONAL_KEY);
+ *	}
+ *	public String getKeyFromData(Tuple2<String, String> data) {
+ *		return data.f0;
+ *	}
+ *	public String getValueFromData(Tuple2<String, String> data) {
+ *		return data.f1;
+ *	}
+ *}
+ *JedisPoolConfig jedisPoolConfig = new JedisPoolConfig.Builder()
+ *    .setHost(REDIS_HOST).setPort(REDIS_PORT).build();
+ *new RedisSink<String>(jedisPoolConfig, new RedisExampleMapper(RedisCommand.LPUSH));
+ *}</pre>
+ *
+ * @param <IN> Type of the elements emitted by this sink
+ */
+public class RedisSink<IN> extends RichSinkFunction<IN> {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(RedisSink.class);
+
+	/**
+	 * This additional key needed for {@link RedisDataType#HASH} and {@link RedisDataType#SORTED_SET}.
+	 * Other {@link RedisDataType} works only with two variable i.e. name of the list and value to be added.
+	 * But for {@link RedisDataType#HASH} and {@link RedisDataType#SORTED_SET} we need three variables.
+	 * <p>For {@link RedisDataType#HASH} we need hash name, hash key and element.
+	 * {@code additionalKey} used as hash name for {@link RedisDataType#HASH}
+	 * <p>For {@link RedisDataType#SORTED_SET} we need set name, the element and it's score.
+	 * {@code additionalKey} used as set name for {@link RedisDataType#SORTED_SET}
+	 */
+	private String additionalKey;
+	private RedisMapper<IN> redisSinkMapper;
+	private RedisCommand redisCommand;
+
+	private FlinkJedisConfigBase flinkJedisConfigBase;
+	private RedisCommandsContainer redisCommandsContainer;
+
+	/**
+	 * Creates a new {@link RedisSink} that connects to the Redis server.
+	 *
+	 * @param flinkJedisConfigBase The configuration of {@link FlinkJedisConfigBase}
+	 * @param redisSinkMapper This is used to generate Redis command and key value from incoming elements.
+	 */
+	public RedisSink(FlinkJedisConfigBase flinkJedisConfigBase, RedisMapper<IN> redisSinkMapper) {
+		Preconditions.checkNotNull(flinkJedisConfigBase, "Redis connection pool config should not be null");
+		Preconditions.checkNotNull(redisSinkMapper, "Redis Mapper can not be null");
+		Preconditions.checkNotNull(redisSinkMapper.getCommandDescription(), "Redis Mapper data type description can not be null");
+
+		this.flinkJedisConfigBase = flinkJedisConfigBase;
+
+		this.redisSinkMapper = redisSinkMapper;
+		RedisCommandDescription redisCommandDescription = redisSinkMapper.getCommandDescription();
+		this.redisCommand = redisCommandDescription.getCommand();
+		this.additionalKey = redisCommandDescription.getAdditionalKey();
+	}
+
+	/**
+	 * Called when new data arrives to the sink, and forwards it to Redis channel.
+	 * Depending on the specified Redis data type (see {@link RedisDataType}),
+	 * a different Redis command will be applied.
+	 * Available commands are RPUSH, LPUSH, SADD, PUBLISH, SET, PFADD, HSET, ZADD.
+	 *
+	 * @param input The incoming data
+	 */
+	@Override
+	public void invoke(IN input) throws Exception {
+		String key = redisSinkMapper.getKeyFromData(input);
+		String value = redisSinkMapper.getValueFromData(input);
+
+		switch (redisCommand) {
+			case RPUSH:
+				this.redisCommandsContainer.rpush(key, value);
+				break;
+			case LPUSH:
+				this.redisCommandsContainer.lpush(key, value);
+				break;
+			case SADD:
+				this.redisCommandsContainer.sadd(key, value);
+				break;
+			case SET:
+				this.redisCommandsContainer.set(key, value);
+				break;
+			case PFADD:
+				this.redisCommandsContainer.pfadd(key, value);
+				break;
+			case PUBLISH:
+				this.redisCommandsContainer.publish(key, value);
+				break;
+			case ZADD:
+				this.redisCommandsContainer.zadd(this.additionalKey, value, key);
+				break;
+			case HSET:
+				this.redisCommandsContainer.hset(this.additionalKey, key, value);
+				break;
+			default:
+				throw new IllegalArgumentException("Cannot process such data type: " + redisCommand);
+		}
+	}
+
+	/**
+	 * Initializes the connection to Redis by either cluster or sentinels or single server.
+	 *
+	 * @throws IllegalArgumentException if jedisPoolConfig, jedisClusterConfig and jedisSentinelConfig are all null
+     */
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		try {
+			this.redisCommandsContainer = RedisCommandsContainerBuilder.build(this.flinkJedisConfigBase);
+			this.redisCommandsContainer.open();
+		} catch (Exception e) {
+			LOG.error("Redis has not been properly initialized: ", e);
+			throw e;
+		}
+	}
+
+	/**
+	 * Closes commands container.
+	 * @throws IOException if command container is unable to close.
+	 */
+	@Override
+	public void close() throws IOException {
+		if (redisCommandsContainer != null) {
+			redisCommandsContainer.close();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java
new file mode 100644
index 0000000..6e6cfe5
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis.common.config;
+
+import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
+import org.apache.flink.util.Preconditions;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.Protocol;
+
+import java.net.InetSocketAddress;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Configuration for Jedis cluster.
+ */
+public class FlinkJedisClusterConfig extends FlinkJedisConfigBase {
+	private static final long serialVersionUID = 1L;
+
+	private final Set<InetSocketAddress> nodes;
+	private final int maxRedirections;
+
+
+	/**
+	 * Jedis cluster configuration.
+	 * The list of node is mandatory, and when nodes is not set, it throws NullPointerException.
+	 *
+	 * @param nodes list of node information for JedisCluster
+	 * @param connectionTimeout socket / connection timeout. The default is 2000
+	 * @param maxRedirections limit of redirections-how much we'll follow MOVED or ASK
+	 * @param maxTotal the maximum number of objects that can be allocated by the pool
+	 * @param maxIdle the cap on the number of "idle" instances in the pool
+	 * @param minIdle the minimum number of idle objects to maintain in the pool
+	 * @throws NullPointerException if parameter {@code nodes} is {@code null}
+	 */
+	private FlinkJedisClusterConfig(Set<InetSocketAddress> nodes, int connectionTimeout, int maxRedirections,
+									int maxTotal, int maxIdle, int minIdle) {
+		super(connectionTimeout, maxTotal, maxIdle, minIdle);
+
+		Preconditions.checkNotNull(nodes, "Node information should be presented");
+		Preconditions.checkArgument(!nodes.isEmpty(), "Redis cluster hosts should not be empty");
+		this.nodes = new HashSet<>(nodes);
+		this.maxRedirections = maxRedirections;
+	}
+
+
+
+	/**
+	 * Returns nodes.
+	 *
+	 * @return list of node information
+	 */
+	public Set<HostAndPort> getNodes() {
+		Set<HostAndPort> ret = new HashSet<>();
+		for (InetSocketAddress node : nodes) {
+			ret.add(new HostAndPort(node.getHostName(), node.getPort()));
+		}
+		return ret;
+	}
+
+	/**
+	 * Returns limit of redirection.
+	 *
+	 * @return limit of redirection
+	 */
+	public int getMaxRedirections() {
+		return maxRedirections;
+	}
+
+
+	/**
+	 * Builder for initializing  {@link FlinkJedisClusterConfig}.
+	 */
+	public static class Builder {
+		private Set<InetSocketAddress> nodes;
+		private int timeout = Protocol.DEFAULT_TIMEOUT;
+		private int maxRedirections = 5;
+		private int maxTotal = GenericObjectPoolConfig.DEFAULT_MAX_TOTAL;
+		private int maxIdle = GenericObjectPoolConfig.DEFAULT_MAX_IDLE;
+		private int minIdle = GenericObjectPoolConfig.DEFAULT_MIN_IDLE;
+
+		/**
+		 * Sets list of node.
+		 *
+		 * @param nodes list of node
+		 * @return Builder itself
+		 */
+		public Builder setNodes(Set<InetSocketAddress> nodes) {
+			this.nodes = nodes;
+			return this;
+		}
+
+		/**
+		 * Sets socket / connection timeout.
+		 *
+		 * @param timeout socket / connection timeout, default value is 2000
+		 * @return Builder itself
+		 */
+		public Builder setTimeout(int timeout) {
+			this.timeout = timeout;
+			return this;
+		}
+
+		/**
+		 * Sets limit of redirection.
+		 *
+		 * @param maxRedirections limit of redirection, default value is 5
+		 * @return Builder itself
+		 */
+		public Builder setMaxRedirections(int maxRedirections) {
+			this.maxRedirections = maxRedirections;
+			return this;
+		}
+
+		/**
+		 * Sets value for the {@code maxTotal} configuration attribute
+		 * for pools to be created with this configuration instance.
+		 *
+		 * @param maxTotal maxTotal the maximum number of objects that can be allocated by the pool, default value is 8
+		 * @return Builder itself
+		 */
+		public Builder setMaxTotal(int maxTotal) {
+			this.maxTotal = maxTotal;
+			return this;
+		}
+
+		/**
+		 * Sets value for the {@code maxIdle} configuration attribute
+		 * for pools to be created with this configuration instance.
+		 *
+		 * @param maxIdle the cap on the number of "idle" instances in the pool, default value is 8
+		 * @return Builder itself
+		 */
+		public Builder setMaxIdle(int maxIdle) {
+			this.maxIdle = maxIdle;
+			return this;
+		}
+
+		/**
+		 * Sets value for the {@code minIdle} configuration attribute
+		 * for pools to be created with this configuration instance.
+		 *
+		 * @param minIdle the minimum number of idle objects to maintain in the pool, default value is 0
+		 * @return Builder itself
+		 */
+		public Builder setMinIdle(int minIdle) {
+			this.minIdle = minIdle;
+			return this;
+		}
+
+		/**
+		 * Builds JedisClusterConfig.
+		 *
+		 * @return JedisClusterConfig
+		 */
+		public FlinkJedisClusterConfig build() {
+			return new FlinkJedisClusterConfig(nodes, timeout, maxRedirections, maxTotal, maxIdle, minIdle);
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "JedisClusterConfig{" +
+			"nodes=" + nodes +
+			", timeout=" + connectionTimeout +
+			", maxRedirections=" + maxRedirections +
+			", maxTotal=" + maxTotal +
+			", maxIdle=" + maxIdle +
+			", minIdle=" + minIdle +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBase.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBase.java
new file mode 100644
index 0000000..a2489b8
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBase.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.redis.common.config;
+
+import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/**
+ * Base class for Flink Redis configuration.
+ */
+public abstract class FlinkJedisConfigBase implements Serializable {
+	private static final long serialVersionUID = 1L;
+
+	protected final int maxTotal;
+	protected final int maxIdle;
+	protected final int minIdle;
+	protected final int connectionTimeout;
+
+	protected FlinkJedisConfigBase(int connectionTimeout, int maxTotal, int maxIdle, int minIdle){
+		Preconditions.checkArgument(connectionTimeout >= 0, "connection timeout can not be negative");
+		Preconditions.checkArgument(maxTotal >= 0, "maxTotal value can not be negative");
+		Preconditions.checkArgument(maxIdle >= 0, "maxIdle value can not be negative");
+		Preconditions.checkArgument(minIdle >= 0, "minIdle value can not be negative");
+		this.connectionTimeout = connectionTimeout;
+		this.maxTotal = maxTotal;
+		this.maxIdle = maxIdle;
+		this.minIdle = minIdle;
+	}
+
+	/**
+	 * Returns timeout.
+	 *
+	 * @return connection timeout
+	 */
+	public int getConnectionTimeout() {
+		return connectionTimeout;
+	}
+
+	/**
+	 * Get the value for the {@code maxTotal} configuration attribute
+	 * for pools to be created with this configuration instance.
+	 *
+	 * @return  The current setting of {@code maxTotal} for this
+	 *          configuration instance
+	 * @see GenericObjectPoolConfig#getMaxTotal()
+	 */
+	public int getMaxTotal() {
+		return maxTotal;
+	}
+
+	/**
+	 * Get the value for the {@code maxIdle} configuration attribute
+	 * for pools to be created with this configuration instance.
+	 *
+	 * @return  The current setting of {@code maxIdle} for this
+	 *          configuration instance
+	 * @see GenericObjectPoolConfig#getMaxIdle()
+	 */
+	public int getMaxIdle() {
+		return maxIdle;
+	}
+
+	/**
+	 * Get the value for the {@code minIdle} configuration attribute
+	 * for pools to be created with this configuration instance.
+	 *
+	 * @return  The current setting of {@code minIdle} for this
+	 *          configuration instance
+	 * @see GenericObjectPoolConfig#getMinIdle()
+	 */
+	public int getMinIdle() {
+		return minIdle;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java
new file mode 100644
index 0000000..d261a35
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis.common.config;
+
+import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
+import org.apache.flink.util.Preconditions;
+import redis.clients.jedis.Protocol;
+
+/**
+ * Configuration for Jedis pool.
+ */
+public class FlinkJedisPoolConfig extends FlinkJedisConfigBase {
+
+	private static final long serialVersionUID = 1L;
+
+	private final String host;
+	private final int port;
+	private final int database;
+	private final String password;
+
+
+	/**
+	 * Jedis pool configuration.
+	 * The host is mandatory, and when host is not set, it throws NullPointerException.
+	 *
+	 * @param host hostname or IP
+	 * @param port port, default value is 6379
+	 * @param connectionTimeout socket / connection timeout, default value is 2000 milli second
+	 * @param password password, if any
+	 * @param database database index
+	 * @param maxTotal the maximum number of objects that can be allocated by the pool, default value is 8
+	 * @param maxIdle the cap on the number of "idle" instances in the pool, default value is 8
+	 * @param minIdle the minimum number of idle objects to maintain in the pool, default value is 0
+	 * @throws NullPointerException if parameter {@code host} is {@code null}
+	 */
+	private FlinkJedisPoolConfig(String host, int port, int connectionTimeout, String password, int database,
+								int maxTotal, int maxIdle, int minIdle) {
+		super(connectionTimeout, maxTotal, maxIdle, minIdle);
+		Preconditions.checkNotNull(host, "Host information should be presented");
+		this.host = host;
+		this.port = port;
+		this.database = database;
+		this.password = password;
+	}
+
+	/**
+	 * Returns host.
+	 *
+	 * @return hostname or IP
+	 */
+	public String getHost() {
+		return host;
+	}
+
+	/**
+	 * Returns port.
+	 *
+	 * @return port
+	 */
+	public int getPort() {
+		return port;
+	}
+
+
+	/**
+	 * Returns database index.
+	 *
+	 * @return database index
+	 */
+	public int getDatabase() {
+		return database;
+	}
+
+	/**
+	 * Returns password.
+	 *
+	 * @return password
+	 */
+	public String getPassword() {
+		return password;
+	}
+
+	/**
+	 * Builder for initializing  {@link FlinkJedisPoolConfig}.
+	 */
+	public static class Builder {
+		private String host;
+		private int port = Protocol.DEFAULT_PORT;
+		private int timeout = Protocol.DEFAULT_TIMEOUT;
+		private int database = Protocol.DEFAULT_DATABASE;
+		private String password;
+		private int maxTotal = GenericObjectPoolConfig.DEFAULT_MAX_TOTAL;
+		private int maxIdle = GenericObjectPoolConfig.DEFAULT_MAX_IDLE;
+		private int minIdle = GenericObjectPoolConfig.DEFAULT_MIN_IDLE;
+
+		/**
+		 * Sets value for the {@code maxTotal} configuration attribute
+		 * for pools to be created with this configuration instance.
+		 *
+		 * @param maxTotal maxTotal the maximum number of objects that can be allocated by the pool, default value is 8
+         * @return Builder itself
+         */
+		public Builder setMaxTotal(int maxTotal) {
+			this.maxTotal = maxTotal;
+			return this;
+		}
+
+		/**
+		 * Sets value for the {@code maxIdle} configuration attribute
+		 * for pools to be created with this configuration instance.
+		 *
+		 * @param maxIdle the cap on the number of "idle" instances in the pool, default value is 8
+         * @return Builder itself
+         */
+		public Builder setMaxIdle(int maxIdle) {
+			this.maxIdle = maxIdle;
+			return this;
+		}
+
+		/**
+		 * Sets value for the {@code minIdle} configuration attribute
+		 * for pools to be created with this configuration instance.
+		 *
+		 * @param minIdle the minimum number of idle objects to maintain in the pool, default value is 0
+         * @return Builder itself
+         */
+		public Builder setMinIdle(int minIdle) {
+			this.minIdle = minIdle;
+			return this;
+		}
+
+		/**
+		 * Sets host.
+		 *
+		 * @param host host
+		 * @return Builder itself
+		 */
+		public Builder setHost(String host) {
+			this.host = host;
+			return this;
+		}
+
+		/**
+		 * Sets port.
+		 *
+		 * @param port port, default value is 6379
+		 * @return Builder itself
+		 */
+		public Builder setPort(int port) {
+			this.port = port;
+			return this;
+		}
+
+		/**
+		 * Sets timeout.
+		 *
+		 * @param timeout timeout, default value is 2000
+		 * @return Builder itself
+		 */
+		public Builder setTimeout(int timeout) {
+			this.timeout = timeout;
+			return this;
+		}
+
+		/**
+		 * Sets database index.
+		 *
+		 * @param database database index, default value is 0
+		 * @return Builder itself
+		 */
+		public Builder setDatabase(int database) {
+			this.database = database;
+			return this;
+		}
+
+		/**
+		 * Sets password.
+		 *
+		 * @param password password, if any
+		 * @return Builder itself
+		 */
+		public Builder setPassword(String password) {
+			this.password = password;
+			return this;
+		}
+
+
+		/**
+		 * Builds JedisPoolConfig.
+		 *
+		 * @return JedisPoolConfig
+		 */
+		public FlinkJedisPoolConfig build() {
+			return new FlinkJedisPoolConfig(host, port, timeout, password, database, maxTotal, maxIdle, minIdle);
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "JedisPoolConfig{" +
+			"host='" + host + '\'' +
+			", port=" + port +
+			", timeout=" + connectionTimeout +
+			", database=" + database +
+			", maxTotal=" + maxTotal +
+			", maxIdle=" + maxIdle +
+			", minIdle=" + minIdle +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java
new file mode 100644
index 0000000..2cdb397
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis.common.config;
+
+import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import redis.clients.jedis.Protocol;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Configuration for Jedis Sentinel pool.
+ */
+public class FlinkJedisSentinelConfig extends FlinkJedisConfigBase {
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkJedisSentinelConfig.class);
+
+	private final String masterName;
+	private final Set<String> sentinels;
+	private final int soTimeout;
+	private final String password;
+	private final int database;
+
+	/**
+	 * Jedis Sentinels config.
+	 * The master name and sentinels are mandatory, and when you didn't set these, it throws NullPointerException.
+	 *
+	 * @param masterName master name of the replica set
+	 * @param sentinels set of sentinel hosts
+	 * @param connectionTimeout timeout connection timeout
+	 * @param soTimeout timeout socket timeout
+	 * @param password password, if any
+	 * @param database database database index
+	 * @param maxTotal maxTotal the maximum number of objects that can be allocated by the pool
+	 * @param maxIdle the cap on the number of "idle" instances in the pool
+	 * @param minIdle the minimum number of idle objects to maintain in the pool
+	 *
+	 * @throws NullPointerException if {@code masterName} or {@code sentinels} is {@code null}
+	 * @throws IllegalArgumentException if {@code sentinels} are empty
+	 */
+	private FlinkJedisSentinelConfig(String masterName, Set<String> sentinels,
+									int connectionTimeout, int soTimeout,
+									String password, int database,
+									int maxTotal, int maxIdle, int minIdle) {
+		super(connectionTimeout, maxTotal, maxIdle, minIdle);
+		Preconditions.checkNotNull(masterName, "Master name should be presented");
+		Preconditions.checkNotNull(sentinels, "Sentinels information should be presented");
+		Preconditions.checkArgument(!sentinels.isEmpty(), "Sentinel hosts should not be empty");
+
+		this.masterName = masterName;
+		this.sentinels = new HashSet<>(sentinels);
+		this.soTimeout = soTimeout;
+		this.password = password;
+		this.database = database;
+	}
+
+	/**
+	 * Returns master name of the replica set.
+	 *
+	 * @return master name of the replica set.
+	 */
+	public String getMasterName() {
+		return masterName;
+	}
+
+	/**
+	 * Returns Sentinels host addresses.
+	 *
+	 * @return Set of Sentinels host addresses
+	 */
+	public Set<String> getSentinels() {
+		return sentinels;
+	}
+
+	/**
+	 * Returns socket timeout.
+	 *
+	 * @return socket timeout
+	 */
+	public int getSoTimeout() {
+		return soTimeout;
+	}
+
+	/**
+	 * Returns password.
+	 *
+	 * @return password
+	 */
+	public String getPassword() {
+		return password;
+	}
+
+	/**
+	 * Returns database index.
+	 *
+	 * @return database index
+	 */
+	public int getDatabase() {
+		return database;
+	}
+
+	/**
+	 * Builder for initializing {@link FlinkJedisSentinelConfig}.
+	 */
+	public static class Builder {
+		private String masterName;
+		private Set<String> sentinels;
+		private int connectionTimeout = Protocol.DEFAULT_TIMEOUT;
+		private int soTimeout = Protocol.DEFAULT_TIMEOUT;
+		private String password;
+		private int database = Protocol.DEFAULT_DATABASE;
+		private int maxTotal = GenericObjectPoolConfig.DEFAULT_MAX_TOTAL;
+		private int maxIdle = GenericObjectPoolConfig.DEFAULT_MAX_IDLE;
+		private int minIdle = GenericObjectPoolConfig.DEFAULT_MIN_IDLE;
+
+		/**
+		 * Sets master name of the replica set.
+		 *
+		 * @param masterName  master name of the replica set
+		 * @return Builder itself
+         */
+		public Builder setMasterName(String masterName) {
+			this.masterName = masterName;
+			return this;
+		}
+
+		/**
+		 * Sets sentinels address.
+		 *
+		 * @param sentinels host set of the sentinels
+		 * @return Builder itself
+         */
+		public Builder setSentinels(Set<String> sentinels) {
+			this.sentinels = sentinels;
+			return this;
+		}
+
+		/**
+		 * Sets connection timeout.
+		 *
+		 * @param connectionTimeout connection timeout, default value is 2000
+		 * @return Builder itself
+		 */
+		public Builder setConnectionTimeout(int connectionTimeout) {
+			this.connectionTimeout = connectionTimeout;
+			return this;
+		}
+
+		/**
+		 * Sets socket timeout.
+		 *
+		 * @param soTimeout socket timeout, default value is 2000
+         * @return Builder itself
+         */
+		public Builder setSoTimeout(int soTimeout) {
+			this.soTimeout = soTimeout;
+			return this;
+		}
+
+		/**
+		 * Sets password.
+		 *
+		 * @param password password, if any
+		 * @return Builder itself
+		 */
+		public Builder setPassword(String password) {
+			this.password = password;
+			return this;
+		}
+
+		/**
+		 * Sets database index.
+		 *
+		 * @param database database index, default value is 0
+		 * @return Builder itself
+		 */
+		public Builder setDatabase(int database) {
+			this.database = database;
+			return this;
+		}
+
+		/**
+		 * Sets value for the {@code maxTotal} configuration attribute
+		 * for pools to be created with this configuration instance.
+		 *
+		 * @param maxTotal maxTotal the maximum number of objects that can be allocated by the pool, default value is 8
+		 * @return Builder itself
+		 */
+		public Builder setMaxTotal(int maxTotal) {
+			this.maxTotal = maxTotal;
+			return this;
+		}
+
+		/**
+		 * Sets value for the {@code maxIdle} configuration attribute
+		 * for pools to be created with this configuration instance.
+		 *
+		 * @param maxIdle the cap on the number of "idle" instances in the pool, default value is 8
+		 * @return Builder itself
+		 */
+		public Builder setMaxIdle(int maxIdle) {
+			this.maxIdle = maxIdle;
+			return this;
+		}
+
+		/**
+		 * Sets value for the {@code minIdle} configuration attribute
+		 * for pools to be created with this configuration instance.
+		 *
+		 * @param minIdle the minimum number of idle objects to maintain in the pool, default value is 0
+		 * @return Builder itself
+		 */
+		public Builder setMinIdle(int minIdle) {
+			this.minIdle = minIdle;
+			return this;
+		}
+
+		/**
+		 * Builds JedisSentinelConfig.
+		 *
+		 * @return JedisSentinelConfig
+		 */
+		public FlinkJedisSentinelConfig build(){
+			return new FlinkJedisSentinelConfig(masterName, sentinels, connectionTimeout, soTimeout,
+				password, database, maxTotal, maxIdle, minIdle);
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "JedisSentinelConfig{" +
+			"masterName='" + masterName + '\'' +
+			", connectionTimeout=" + connectionTimeout +
+			", soTimeout=" + soTimeout +
+			", database=" + database +
+			", maxTotal=" + maxTotal +
+			", maxIdle=" + maxIdle +
+			", minIdle=" + minIdle +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisClusterContainer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisClusterContainer.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisClusterContainer.java
new file mode 100644
index 0000000..d6621d6
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisClusterContainer.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis.common.container;
+
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import redis.clients.jedis.JedisCluster;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Redis command container if we want to connect to a Redis cluster.
+ */
+public class RedisClusterContainer implements RedisCommandsContainer, Closeable {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(RedisClusterContainer.class);
+
+	private transient JedisCluster jedisCluster;
+
+	/**
+	 * Initialize Redis command container for Redis cluster.
+	 *
+	 * @param jedisCluster JedisCluster instance
+	 */
+	public RedisClusterContainer(JedisCluster jedisCluster) {
+		Preconditions.checkNotNull(jedisCluster, "Jedis cluster can not be null");
+
+		this.jedisCluster = jedisCluster;
+	}
+
+	@Override
+	public void open() throws Exception {
+
+		// echo() tries to open a connection and echos back the
+		// message passed as argument. Here we use it to monitor
+		// if we can communicate with the cluster.
+
+		jedisCluster.echo("Test");
+	}
+
+	@Override
+	public void hset(final String key, final String hashField, final String value) {
+		try {
+			jedisCluster.hset(key, hashField, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command HSET to hash {} error message {}",
+					key, hashField, e.getMessage());
+			}
+			throw e;
+		}
+	}
+
+	@Override
+	public void rpush(final String listName, final String value) {
+		try {
+			jedisCluster.rpush(listName, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command RPUSH to list {} error message: {}",
+					listName, e.getMessage());
+			}
+			throw e;
+		}
+	}
+
+	@Override
+	public void lpush(String listName, String value) {
+		try {
+			jedisCluster.lpush(listName, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command LPUSH to list {} error message: {}",
+					listName, e.getMessage());
+			}
+			throw e;
+		}
+	}
+
+	@Override
+	public void sadd(final String setName, final String value) {
+		try {
+			jedisCluster.sadd(setName, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command RPUSH to set {} error message {}",
+					setName, e.getMessage());
+			}
+			throw e;
+		}
+	}
+
+	@Override
+	public void publish(final String channelName, final String message) {
+		try {
+			jedisCluster.publish(channelName, message);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command PUBLISH to channel {} error message {}",
+					channelName, e.getMessage());
+			}
+			throw e;
+		}
+	}
+
+	@Override
+	public void set(final String key, final String value) {
+		try {
+			jedisCluster.set(key, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command SET to key {} error message {}",
+					key, e.getMessage());
+			}
+			throw e;
+		}
+	}
+
+	@Override
+	public void pfadd(final String key, final String element) {
+		try {
+			jedisCluster.set(key, element);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command PFADD to key {} error message {}",
+					key, e.getMessage());
+			}
+			throw e;
+		}
+	}
+
+	@Override
+	public void zadd(final String key, final String score, final String element) {
+		try {
+			jedisCluster.zadd(key, Double.valueOf(score), element);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command ZADD to set {} error message {}",
+					key, e.getMessage());
+			}
+			throw e;
+		}
+	}
+
+	/**
+	 * Closes the {@link JedisCluster}.
+	 */
+	@Override
+	public void close() throws IOException {
+		this.jedisCluster.close();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainer.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainer.java
new file mode 100644
index 0000000..55dbfc2
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainer.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.redis.common.container;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * The container for all available Redis commands.
+ */
+public interface RedisCommandsContainer extends Serializable {
+
+	/**
+	 * Open the Jedis container.
+	 *
+	 * @throws Exception if the instance can not be opened properly
+	 */
+	void open() throws Exception;
+
+	/**
+	 * Sets field in the hash stored at key to value.
+	 * If key does not exist, a new key holding a hash is created.
+	 * If field already exists in the hash, it is overwritten.
+	 *
+	 * @param key Hash name
+	 * @param hashField Hash field
+	 * @param value Hash value
+	 */
+	void hset(String key, String hashField, String value);
+
+	/**
+	 * Insert the specified value at the tail of the list stored at key.
+	 * If key does not exist, it is created as empty list before performing the push operation.
+	 *
+	 * @param listName Name of the List
+	 * @param value  Value to be added
+	 */
+	void rpush(String listName, String value);
+
+	/**
+	 * Insert the specified value at the head of the list stored at key.
+	 * If key does not exist, it is created as empty list before performing the push operation.
+	 *
+	 * @param listName Name of the List
+	 * @param value  Value to be added
+	 */
+	void lpush(String listName, String value);
+
+	/**
+	 * Add the specified member to the set stored at key.
+	 * Specified members that are already a member of this set are ignored.
+	 * If key does not exist, a new set is created before adding the specified members.
+	 *
+	 * @param setName Name of the Set
+	 * @param value Value to be added
+	 */
+	void sadd(String setName, String value);
+
+	/**
+	 * Posts a message to the given channel.
+	 *
+	 * @param channelName Name of the channel to which data will be published
+	 * @param message the message
+	 */
+	void publish(String channelName, String message);
+
+	/**
+	 * Set key to hold the string value. If key already holds a value, it is overwritten,
+	 * regardless of its type. Any previous time to live associated with the key is
+	 * discarded on successful SET operation.
+	 *
+	 * @param key the key name in which value to be set
+	 * @param value the value
+	 */
+	void set(String key, String value);
+
+	/**
+	 * Adds all the element arguments to the HyperLogLog data structure
+	 * stored at the variable name specified as first argument.
+	 *
+	 * @param key The name of the key
+	 * @param element the element
+	 */
+	void pfadd(String key, String element);
+
+	/**
+	 * Adds the specified member with the specified scores to the sorted set stored at key.
+	 *
+	 * @param key The name of the Sorted Set
+	 * @param score Score of the element
+	 * @param element  element to be added
+	 */
+	void zadd(String key, String score, String element);
+
+	/**
+	 * Close the Jedis container.
+	 *
+	 * @throws IOException if the instance can not be closed properly
+	 */
+	void close() throws IOException;
+}


[19/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java
new file mode 100644
index 0000000..2ed2f8c
--- /dev/null
+++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.java.io.jdbc.split;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.java.io.jdbc.JDBCInputFormat;
+
+/** 
+ * 
+ * This splits generator actually does nothing but wrapping the query parameters
+ * computed by the user before creating the {@link JDBCInputFormat} instance.
+ * 
+ * */
+public class GenericParameterValuesProvider implements ParameterValuesProvider {
+
+	private final Serializable[][] parameters;
+	
+	public GenericParameterValuesProvider(Serializable[][] parameters) {
+		this.parameters = parameters;
+	}
+
+	@Override
+	public Serializable[][] getParameterValues(){
+		//do nothing...precomputed externally
+		return parameters;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java
new file mode 100644
index 0000000..ac56b98
--- /dev/null
+++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.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.api.java.io.jdbc.split;
+
+import java.io.Serializable;
+
+/** 
+ * 
+ * This query generator assumes that the query to parameterize contains a BETWEEN constraint on a numeric column.
+ * The generated query set will be of size equal to the configured fetchSize (apart the last one range),
+ * ranging from the min value up to the max.
+ * 
+ * For example, if there's a table <CODE>BOOKS</CODE> with a numeric PK <CODE>id</CODE>, using a query like:
+ * <PRE>
+ *   SELECT * FROM BOOKS WHERE id BETWEEN ? AND ?
+ * </PRE>
+ *
+ * you can use this class to automatically generate the parameters of the BETWEEN clause,
+ * based on the passed constructor parameters.
+ * 
+ * */
+public class NumericBetweenParametersProvider implements ParameterValuesProvider {
+
+	private long fetchSize;
+	private final long min;
+	private final long max;
+	
+	public NumericBetweenParametersProvider(long fetchSize, long min, long max) {
+		this.fetchSize = fetchSize;
+		this.min = min;
+		this.max = max;
+	}
+
+	@Override
+	public Serializable[][] getParameterValues(){
+		double maxElemCount = (max - min) + 1;
+		int size = new Double(Math.ceil(maxElemCount / fetchSize)).intValue();
+		Serializable[][] parameters = new Serializable[size][2];
+		int count = 0;
+		for (long i = min; i < max; i += fetchSize, count++) {
+			long currentLimit = i + fetchSize - 1;
+			parameters[count] = new Long[]{i,currentLimit};
+			if (currentLimit + 1 + fetchSize > max) {
+				parameters[count + 1] = new Long[]{currentLimit + 1, max};
+				break;
+			}
+		}
+		return parameters;
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java
new file mode 100644
index 0000000..c194497
--- /dev/null
+++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.jdbc.split;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.java.io.jdbc.JDBCInputFormat;
+
+/**
+ * 
+ * This interface is used by the {@link JDBCInputFormat} to compute the list of parallel query to run (i.e. splits).
+ * Each query will be parameterized using a row of the matrix provided by each {@link ParameterValuesProvider} implementation
+ * 
+ * */
+public interface ParameterValuesProvider {
+
+	/** Returns the necessary parameters array to use for query in parallel a table */
+	public Serializable[][] getParameterValues();
+	
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java
new file mode 100644
index 0000000..da9469b
--- /dev/null
+++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.io.jdbc;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.jdbc.JDBCInputFormat.JDBCInputFormatBuilder;
+import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider;
+import org.apache.flink.api.table.Row;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class JDBCFullTest extends JDBCTestBase {
+
+	@Test
+	public void testJdbcInOut() throws Exception {
+		//run without parallelism
+		runTest(false);
+
+		//cleanup
+		JDBCTestBase.tearDownClass();
+		JDBCTestBase.prepareTestDb();
+		
+		//run expliting parallelism
+		runTest(true);
+		
+	}
+
+	private void runTest(boolean exploitParallelism) {
+		ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment();
+		JDBCInputFormatBuilder inputBuilder = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername(JDBCTestBase.DRIVER_CLASS)
+				.setDBUrl(JDBCTestBase.DB_URL)
+				.setQuery(JDBCTestBase.SELECT_ALL_BOOKS)
+				.setRowTypeInfo(rowTypeInfo);
+
+		if(exploitParallelism) {
+			final int fetchSize = 1;
+			final Long min = new Long(JDBCTestBase.testData[0][0].toString());
+			final Long max = new Long(JDBCTestBase.testData[JDBCTestBase.testData.length - fetchSize][0].toString());
+			//use a "splittable" query to exploit parallelism
+			inputBuilder = inputBuilder
+					.setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_ID)
+					.setParametersProvider(new NumericBetweenParametersProvider(fetchSize, min, max));
+		}
+		DataSet<Row> source = environment.createInput(inputBuilder.finish());
+
+		//NOTE: in this case (with Derby driver) setSqlTypes could be skipped, but
+		//some database, doens't handle correctly null values when no column type specified
+		//in PreparedStatement.setObject (see its javadoc for more details)
+		source.output(JDBCOutputFormat.buildJDBCOutputFormat()
+				.setDrivername(JDBCTestBase.DRIVER_CLASS)
+				.setDBUrl(JDBCTestBase.DB_URL)
+				.setQuery("insert into newbooks (id,title,author,price,qty) values (?,?,?,?,?)")
+				.setSqlTypes(new int[]{Types.INTEGER, Types.VARCHAR, Types.VARCHAR,Types.DOUBLE,Types.INTEGER})
+				.finish());
+		try {
+			environment.execute();
+		} catch (Exception e) {
+			Assert.fail("JDBC full test failed. " + e.getMessage());
+		}
+
+		try (
+			Connection dbConn = DriverManager.getConnection(JDBCTestBase.DB_URL);
+			PreparedStatement statement = dbConn.prepareStatement(JDBCTestBase.SELECT_ALL_NEWBOOKS);
+			ResultSet resultSet = statement.executeQuery()
+		) {
+			int count = 0;
+			while (resultSet.next()) {
+				count++;
+			}
+			Assert.assertEquals(JDBCTestBase.testData.length, count);
+		} catch (SQLException e) {
+			Assert.fail("JDBC full test failed. " + e.getMessage());
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
new file mode 100644
index 0000000..efae076
--- /dev/null
+++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
@@ -0,0 +1,247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.io.jdbc;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.sql.ResultSet;
+
+import org.apache.flink.api.java.io.jdbc.split.GenericParameterValuesProvider;
+import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider;
+import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.core.io.InputSplit;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class JDBCInputFormatTest extends JDBCTestBase {
+
+	private JDBCInputFormat jdbcInputFormat;
+
+	@After
+	public void tearDown() throws IOException {
+		if (jdbcInputFormat != null) {
+			jdbcInputFormat.close();
+		}
+		jdbcInputFormat = null;
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testUntypedRowInfo() throws IOException {
+		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername("org.apache.derby.jdbc.idontexist")
+				.setDBUrl(DB_URL)
+				.setQuery(SELECT_ALL_BOOKS)
+				.finish();
+		jdbcInputFormat.openInputFormat();
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testInvalidDriver() throws IOException {
+		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername("org.apache.derby.jdbc.idontexist")
+				.setDBUrl(DB_URL)
+				.setQuery(SELECT_ALL_BOOKS)
+				.setRowTypeInfo(rowTypeInfo)
+				.finish();
+		jdbcInputFormat.openInputFormat();
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testInvalidURL() throws IOException {
+		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl("jdbc:der:iamanerror:mory:ebookshop")
+				.setQuery(SELECT_ALL_BOOKS)
+				.setRowTypeInfo(rowTypeInfo)
+				.finish();
+		jdbcInputFormat.openInputFormat();
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testInvalidQuery() throws IOException {
+		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl(DB_URL)
+				.setQuery("iamnotsql")
+				.setRowTypeInfo(rowTypeInfo)
+				.finish();
+		jdbcInputFormat.openInputFormat();
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testIncompleteConfiguration() throws IOException {
+		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setQuery(SELECT_ALL_BOOKS)
+				.setRowTypeInfo(rowTypeInfo)
+				.finish();
+	}
+
+	@Test
+	public void testJDBCInputFormatWithoutParallelism() throws IOException, InstantiationException, IllegalAccessException {
+		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl(DB_URL)
+				.setQuery(SELECT_ALL_BOOKS)
+				.setRowTypeInfo(rowTypeInfo)
+				.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
+				.finish();
+		//this query does not exploit parallelism
+		Assert.assertEquals(1, jdbcInputFormat.createInputSplits(1).length);
+		jdbcInputFormat.openInputFormat();
+		jdbcInputFormat.open(null);
+		Row row =  new Row(5);
+		int recordCount = 0;
+		while (!jdbcInputFormat.reachedEnd()) {
+			Row next = jdbcInputFormat.nextRecord(row);
+			if (next == null) {
+				break;
+			}
+			
+			if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());}
+			if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());}
+			if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());}
+			if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());}
+			if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());}
+
+			for (int x = 0; x < 5; x++) {
+				if(testData[recordCount][x]!=null) {
+					Assert.assertEquals(testData[recordCount][x], next.productElement(x));
+				}
+			}
+			recordCount++;
+		}
+		jdbcInputFormat.close();
+		jdbcInputFormat.closeInputFormat();
+		Assert.assertEquals(testData.length, recordCount);
+	}
+	
+	@Test
+	public void testJDBCInputFormatWithParallelismAndNumericColumnSplitting() throws IOException, InstantiationException, IllegalAccessException {
+		final int fetchSize = 1;
+		final Long min = new Long(JDBCTestBase.testData[0][0] + "");
+		final Long max = new Long(JDBCTestBase.testData[JDBCTestBase.testData.length - fetchSize][0] + "");
+		ParameterValuesProvider pramProvider = new NumericBetweenParametersProvider(fetchSize, min, max);
+		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl(DB_URL)
+				.setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_ID)
+				.setRowTypeInfo(rowTypeInfo)
+				.setParametersProvider(pramProvider)
+				.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
+				.finish();
+
+		jdbcInputFormat.openInputFormat();
+		InputSplit[] splits = jdbcInputFormat.createInputSplits(1);
+		//this query exploit parallelism (1 split for every id)
+		Assert.assertEquals(testData.length, splits.length);
+		int recordCount = 0;
+		Row row =  new Row(5);
+		for (int i = 0; i < splits.length; i++) {
+			jdbcInputFormat.open(splits[i]);
+			while (!jdbcInputFormat.reachedEnd()) {
+				Row next = jdbcInputFormat.nextRecord(row);
+				if (next == null) {
+					break;
+				}
+				if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());}
+				if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());}
+				if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());}
+				if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());}
+				if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());}
+
+				for (int x = 0; x < 5; x++) {
+					if(testData[recordCount][x]!=null) {
+						Assert.assertEquals(testData[recordCount][x], next.productElement(x));
+					}
+				}
+				recordCount++;
+			}
+			jdbcInputFormat.close();
+		}
+		jdbcInputFormat.closeInputFormat();
+		Assert.assertEquals(testData.length, recordCount);
+	}
+	
+	@Test
+	public void testJDBCInputFormatWithParallelismAndGenericSplitting() throws IOException, InstantiationException, IllegalAccessException {
+		Serializable[][] queryParameters = new String[2][1];
+		queryParameters[0] = new String[]{"Kumar"};
+		queryParameters[1] = new String[]{"Tan Ah Teck"};
+		ParameterValuesProvider paramProvider = new GenericParameterValuesProvider(queryParameters);
+		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl(DB_URL)
+				.setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR)
+				.setRowTypeInfo(rowTypeInfo)
+				.setParametersProvider(paramProvider)
+				.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
+				.finish();
+		jdbcInputFormat.openInputFormat();
+		InputSplit[] splits = jdbcInputFormat.createInputSplits(1);
+		//this query exploit parallelism (1 split for every queryParameters row)
+		Assert.assertEquals(queryParameters.length, splits.length);
+		int recordCount = 0;
+		Row row =  new Row(5);
+		for (int i = 0; i < splits.length; i++) {
+			jdbcInputFormat.open(splits[i]);
+			while (!jdbcInputFormat.reachedEnd()) {
+				Row next = jdbcInputFormat.nextRecord(row);
+				if (next == null) {
+					break;
+				}
+				if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());}
+				if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());}
+				if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());}
+				if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());}
+				if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());}
+
+				recordCount++;
+			}
+			jdbcInputFormat.close();
+		}
+		Assert.assertEquals(3, recordCount);
+		jdbcInputFormat.closeInputFormat();
+	}
+	
+	@Test
+	public void testEmptyResults() throws IOException, InstantiationException, IllegalAccessException {
+		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl(DB_URL)
+				.setQuery(SELECT_EMPTY)
+				.setRowTypeInfo(rowTypeInfo)
+				.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
+				.finish();
+		jdbcInputFormat.openInputFormat();
+		jdbcInputFormat.open(null);
+		Row row = new Row(5);
+		int recordsCnt = 0;
+		while (!jdbcInputFormat.reachedEnd()) {
+			Assert.assertNull(jdbcInputFormat.nextRecord(row));
+			recordsCnt++;
+		}
+		jdbcInputFormat.close();
+		jdbcInputFormat.closeInputFormat();
+		Assert.assertEquals(0, recordsCnt);
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java
new file mode 100644
index 0000000..086a84c
--- /dev/null
+++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.jdbc;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.table.Row;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class JDBCOutputFormatTest extends JDBCTestBase {
+
+	private JDBCOutputFormat jdbcOutputFormat;
+	private Tuple5<Integer, String, String, Double, String> tuple5 = new Tuple5<>();
+
+	@After
+	public void tearDown() throws IOException {
+		if (jdbcOutputFormat != null) {
+			jdbcOutputFormat.close();
+		}
+		jdbcOutputFormat = null;
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testInvalidDriver() throws IOException {
+		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
+				.setDrivername("org.apache.derby.jdbc.idontexist")
+				.setDBUrl(DB_URL)
+				.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
+				.finish();
+		jdbcOutputFormat.open(0, 1);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testInvalidURL() throws IOException {
+		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl("jdbc:der:iamanerror:mory:ebookshop")
+				.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
+				.finish();
+		jdbcOutputFormat.open(0, 1);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testInvalidQuery() throws IOException {
+		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl(DB_URL)
+				.setQuery("iamnotsql")
+				.finish();
+		jdbcOutputFormat.open(0, 1);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testIncompleteConfiguration() throws IOException {
+		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
+				.finish();
+	}
+
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testIncompatibleTypes() throws IOException {
+		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl(DB_URL)
+				.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
+				.finish();
+		jdbcOutputFormat.open(0, 1);
+
+		tuple5.setField(4, 0);
+		tuple5.setField("hello", 1);
+		tuple5.setField("world", 2);
+		tuple5.setField(0.99, 3);
+		tuple5.setField("imthewrongtype", 4);
+
+		Row row = new Row(tuple5.getArity());
+		for (int i = 0; i < tuple5.getArity(); i++) {
+			row.setField(i, tuple5.getField(i));
+		}
+		jdbcOutputFormat.writeRecord(row);
+		jdbcOutputFormat.close();
+	}
+
+	@Test
+	public void testJDBCOutputFormat() throws IOException, InstantiationException, IllegalAccessException {
+
+		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
+				.setDrivername(DRIVER_CLASS)
+				.setDBUrl(DB_URL)
+				.setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE))
+				.finish();
+		jdbcOutputFormat.open(0, 1);
+
+		for (int i = 0; i < testData.length; i++) {
+			Row row = new Row(testData[i].length);
+			for (int j = 0; j < testData[i].length; j++) {
+				row.setField(j, testData[i][j]);
+			}
+			jdbcOutputFormat.writeRecord(row);
+		}
+
+		jdbcOutputFormat.close();
+
+		try (
+			Connection dbConn = DriverManager.getConnection(JDBCTestBase.DB_URL);
+			PreparedStatement statement = dbConn.prepareStatement(JDBCTestBase.SELECT_ALL_NEWBOOKS);
+			ResultSet resultSet = statement.executeQuery()
+		) {
+			int recordCount = 0;
+			while (resultSet.next()) {
+				Row row = new Row(tuple5.getArity());
+				for (int i = 0; i < tuple5.getArity(); i++) {
+					row.setField(i, resultSet.getObject(i + 1));
+				}
+				if (row.productElement(0) != null) {
+					Assert.assertEquals("Field 0 should be int", Integer.class, row.productElement(0).getClass());
+				}
+				if (row.productElement(1) != null) {
+					Assert.assertEquals("Field 1 should be String", String.class, row.productElement(1).getClass());
+				}
+				if (row.productElement(2) != null) {
+					Assert.assertEquals("Field 2 should be String", String.class, row.productElement(2).getClass());
+				}
+				if (row.productElement(3) != null) {
+					Assert.assertEquals("Field 3 should be float", Double.class, row.productElement(3).getClass());
+				}
+				if (row.productElement(4) != null) {
+					Assert.assertEquals("Field 4 should be int", Integer.class, row.productElement(4).getClass());
+				}
+
+				for (int x = 0; x < tuple5.getArity(); x++) {
+					if (JDBCTestBase.testData[recordCount][x] != null) {
+						Assert.assertEquals(JDBCTestBase.testData[recordCount][x], row.productElement(x));
+					}
+				}
+
+				recordCount++;
+			}
+			Assert.assertEquals(JDBCTestBase.testData.length, recordCount);
+		} catch (SQLException e) {
+			Assert.fail("JDBC OutputFormat test failed. " + e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java
new file mode 100644
index 0000000..69ad693
--- /dev/null
+++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.jdbc;
+
+import java.io.OutputStream;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.typeutils.RowTypeInfo;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+
+/**
+ * Base test class for JDBC Input and Output formats
+ */
+public class JDBCTestBase {
+	
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:ebookshop";
+	public static final String INPUT_TABLE = "books";
+	public static final String OUTPUT_TABLE = "newbooks";
+	public static final String SELECT_ALL_BOOKS = "select * from " + INPUT_TABLE;
+	public static final String SELECT_ALL_NEWBOOKS = "select * from " + OUTPUT_TABLE;
+	public static final String SELECT_EMPTY = "select * from books WHERE QTY < 0";
+	public static final String INSERT_TEMPLATE = "insert into %s (id, title, author, price, qty) values (?,?,?,?,?)";
+	public static final String SELECT_ALL_BOOKS_SPLIT_BY_ID = JDBCTestBase.SELECT_ALL_BOOKS + " WHERE id BETWEEN ? AND ?";
+	public static final String SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR = JDBCTestBase.SELECT_ALL_BOOKS + " WHERE author = ?";
+	
+	protected static Connection conn;
+
+	public static final Object[][] testData = {
+			{1001, ("Java public for dummies"), ("Tan Ah Teck"), 11.11, 11},
+			{1002, ("More Java for dummies"), ("Tan Ah Teck"), 22.22, 22},
+			{1003, ("More Java for more dummies"), ("Mohammad Ali"), 33.33, 33},
+			{1004, ("A Cup of Java"), ("Kumar"), 44.44, 44},
+			{1005, ("A Teaspoon of Java"), ("Kevin Jones"), 55.55, 55},
+			{1006, ("A Teaspoon of Java 1.4"), ("Kevin Jones"), 66.66, 66},
+			{1007, ("A Teaspoon of Java 1.5"), ("Kevin Jones"), 77.77, 77},
+			{1008, ("A Teaspoon of Java 1.6"), ("Kevin Jones"), 88.88, 88},
+			{1009, ("A Teaspoon of Java 1.7"), ("Kevin Jones"), 99.99, 99},
+			{1010, ("A Teaspoon of Java 1.8"), ("Kevin Jones"), null, 1010}};
+
+	public static final TypeInformation<?>[] fieldTypes = new TypeInformation<?>[] {
+		BasicTypeInfo.INT_TYPE_INFO,
+		BasicTypeInfo.STRING_TYPE_INFO,
+		BasicTypeInfo.STRING_TYPE_INFO,
+		BasicTypeInfo.DOUBLE_TYPE_INFO,
+		BasicTypeInfo.INT_TYPE_INFO
+	};
+	
+	public static final RowTypeInfo rowTypeInfo = new RowTypeInfo(fieldTypes);
+
+	public static String getCreateQuery(String tableName) {
+		StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE ");
+		sqlQueryBuilder.append(tableName).append(" (");
+		sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,");
+		sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,");
+		sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,");
+		sqlQueryBuilder.append("price FLOAT DEFAULT NULL,");
+		sqlQueryBuilder.append("qty INT DEFAULT NULL,");
+		sqlQueryBuilder.append("PRIMARY KEY (id))");
+		return sqlQueryBuilder.toString();
+	}
+	
+	public static String getInsertQuery() {
+		StringBuilder sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES ");
+		for (int i = 0; i < JDBCTestBase.testData.length; i++) {
+			sqlQueryBuilder.append("(")
+			.append(JDBCTestBase.testData[i][0]).append(",'")
+			.append(JDBCTestBase.testData[i][1]).append("','")
+			.append(JDBCTestBase.testData[i][2]).append("',")
+			.append(JDBCTestBase.testData[i][3]).append(",")
+			.append(JDBCTestBase.testData[i][4]).append(")");
+			if (i < JDBCTestBase.testData.length - 1) {
+				sqlQueryBuilder.append(",");
+			}
+		}
+		String insertQuery = sqlQueryBuilder.toString();
+		return insertQuery;
+	}
+	
+	public static final OutputStream DEV_NULL = new OutputStream() {
+		@Override
+		public void write(int b) {
+		}
+	};
+
+	public static void prepareTestDb() throws Exception {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+		Class.forName(DRIVER_CLASS);
+		Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+
+		//create input table
+		Statement stat = conn.createStatement();
+		stat.executeUpdate(getCreateQuery(INPUT_TABLE));
+		stat.close();
+
+		//create output table
+		stat = conn.createStatement();
+		stat.executeUpdate(getCreateQuery(OUTPUT_TABLE));
+		stat.close();
+
+		//prepare input data
+		stat = conn.createStatement();
+		stat.execute(JDBCTestBase.getInsertQuery());
+		stat.close();
+
+		conn.close();
+	}
+
+	@BeforeClass
+	public static void setUpClass() throws SQLException {
+		try {
+			System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+			prepareDerbyDatabase();
+		} catch (ClassNotFoundException e) {
+			e.printStackTrace();
+			Assert.fail();
+		}
+	}
+
+	private static void prepareDerbyDatabase() throws ClassNotFoundException, SQLException {
+		Class.forName(DRIVER_CLASS);
+		conn = DriverManager.getConnection(DB_URL + ";create=true");
+		createTable(INPUT_TABLE);
+		createTable(OUTPUT_TABLE);
+		insertDataIntoInputTable();
+		conn.close();
+	}
+	
+	private static void createTable(String tableName) throws SQLException {
+		Statement stat = conn.createStatement();
+		stat.executeUpdate(getCreateQuery(tableName));
+		stat.close();
+	}
+	
+	private static void insertDataIntoInputTable() throws SQLException {
+		Statement stat = conn.createStatement();
+		stat.execute(JDBCTestBase.getInsertQuery());
+		stat.close();
+	}
+
+	@AfterClass
+	public static void tearDownClass() {
+		cleanUpDerbyDatabases();
+	}
+
+	private static void cleanUpDerbyDatabases() {
+		try {
+			Class.forName(DRIVER_CLASS);
+			conn = DriverManager.getConnection(DB_URL + ";create=true");
+			Statement stat = conn.createStatement();
+			stat.executeUpdate("DROP TABLE "+INPUT_TABLE);
+			stat.executeUpdate("DROP TABLE "+OUTPUT_TABLE);
+			stat.close();
+			conn.close();
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail();
+		}
+	}
+
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml
new file mode 100644
index 0000000..dcb33eb
--- /dev/null
+++ b/flink-connectors/pom.xml
@@ -0,0 +1,75 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+
+	<artifactId>flink-connectors</artifactId>
+	<name>flink-connectors</name>
+	<packaging>pom</packaging>
+
+	<modules>
+		<module>flink-avro</module>
+		<module>flink-jdbc</module>
+		<module>flink-hadoop-compatibility</module>
+		<module>flink-hbase</module>
+		<module>flink-hcatalog</module>
+		<module>flink-connector-flume</module>
+		<module>flink-connector-kafka-base</module>
+		<module>flink-connector-kafka-0.8</module>
+		<module>flink-connector-kafka-0.9</module>
+		<module>flink-connector-kafka-0.10</module>
+		<module>flink-connector-elasticsearch</module>
+		<module>flink-connector-elasticsearch2</module>
+		<module>flink-connector-rabbitmq</module>
+		<module>flink-connector-twitter</module>
+		<module>flink-connector-nifi</module>
+		<module>flink-connector-cassandra</module>
+		<module>flink-connector-redis</module>
+		<module>flink-connector-filesystem</module>
+	</modules>
+
+	<!-- See main pom.xml for explanation of profiles -->
+	<profiles>
+		<!--
+			We include the kinesis module only optionally because it contains a dependency
+			licenced under the "Amazon Software License".
+			In accordance with the discussion in https://issues.apache.org/jira/browse/LEGAL-198
+			this is an optional module for Flink.
+		-->
+		<profile>
+			<id>include-kinesis</id>
+			<modules>
+				<module>flink-connector-kinesis</module>
+			</modules>
+		</profile>
+	</profiles>
+	
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/pom.xml b/flink-streaming-connectors/flink-connector-cassandra/pom.xml
deleted file mode 100644
index 3a1731c..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/pom.xml
+++ /dev/null
@@ -1,179 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-cassandra_2.10</artifactId>
-	<name>flink-connector-cassandra</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<cassandra.version>2.2.5</cassandra.version>
-		<driver.version>3.0.0</driver.version>
-	</properties>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<reuseForks>true</reuseForks>
-					<forkCount>1</forkCount>
-					<argLine>-Xms256m -Xmx2800m -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
-				</configuration>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<version>2.4.1</version>
-				<executions>
-					<!-- Run shade goal on package phase -->
-					<execution>
-						<id>shade-flink</id>
-						<phase>package</phase>
-						<goals>
-							<goal>shade</goal>
-						</goals>
-						<configuration combine.self="override">
-							<dependencyReducedPomLocation>${project.basedir}/target/dependency-reduced-pom.xml</dependencyReducedPomLocation>
-							<artifactSet>
-								<includes>
-									<include>com.datastax.cassandra:cassandra-driver-core</include>
-									<include>com.datastax.cassandra:cassandra-driver-mapping</include>
-									<include>com.google.guava:guava</include>
-								</includes>
-							</artifactSet>
-							<relocations>
-								<relocation>
-									<pattern>com.google</pattern>
-									<shadedPattern>org.apache.flink.cassandra.shaded.com.google</shadedPattern>
-									<excludes>
-										<exclude>com.google.protobuf.**</exclude>
-										<exclude>com.google.inject.**</exclude>
-									</excludes>
-								</relocation>
-							</relocations>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-		<dependency>
-			<groupId>com.datastax.cassandra</groupId>
-			<artifactId>cassandra-driver-core</artifactId>
-			<version>${driver.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>log4j-over-slf4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.logback</groupId>
-					<artifactId>logback-classic</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-		<dependency>
-			<groupId>com.datastax.cassandra</groupId>
-			<artifactId>cassandra-driver-mapping</artifactId>
-			<version>${driver.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>log4j-over-slf4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.logback</groupId>
-					<artifactId>logback-classic</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.cassandra</groupId>
-			<artifactId>cassandra-all</artifactId>
-			<version>${cassandra.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>log4j-over-slf4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.logback</groupId>
-					<artifactId>logback-classic</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-	</dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java b/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java
deleted file mode 100644
index 849e023..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.batch.connectors.cassandra;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Row;
-import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
-import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
-import org.apache.flink.api.common.io.NonParallelInput;
-import org.apache.flink.api.common.io.RichInputFormat;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
-import org.apache.flink.util.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * InputFormat to read data from Apache Cassandra and generate ${@link Tuple}.
- *
- * @param <OUT> type of Tuple
- */
-public class CassandraInputFormat<OUT extends Tuple> extends RichInputFormat<OUT, InputSplit> implements NonParallelInput {
-	private static final Logger LOG = LoggerFactory.getLogger(CassandraInputFormat.class);
-
-	private final String query;
-	private final ClusterBuilder builder;
-
-	private transient Cluster cluster;
-	private transient Session session;
-	private transient ResultSet resultSet;
-
-	public CassandraInputFormat(String query, ClusterBuilder builder) {
-		Preconditions.checkArgument(!Strings.isNullOrEmpty(query), "Query cannot be null or empty");
-		Preconditions.checkArgument(builder != null, "Builder cannot be null");
-
-		this.query = query;
-		this.builder = builder;
-	}
-
-	@Override
-	public void configure(Configuration parameters) {
-		this.cluster = builder.getCluster();
-	}
-
-	@Override
-	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
-		return cachedStatistics;
-	}
-
-	/**
-	 * Opens a Session and executes the query.
-	 *
-	 * @param ignored
-	 * @throws IOException
-	 */
-	@Override
-	public void open(InputSplit ignored) throws IOException {
-		this.session = cluster.connect();
-		this.resultSet = session.execute(query);
-	}
-
-	@Override
-	public boolean reachedEnd() throws IOException {
-		return resultSet.isExhausted();
-	}
-
-	@Override
-	public OUT nextRecord(OUT reuse) throws IOException {
-		final Row item = resultSet.one();
-		for (int i = 0; i < reuse.getArity(); i++) {
-			reuse.setField(item.getObject(i), i);
-		}
-		return reuse;
-	}
-
-	@Override
-	public InputSplit[] createInputSplits(int minNumSplits) throws IOException {
-		GenericInputSplit[] split = {new GenericInputSplit(0, 1)};
-		return split;
-	}
-
-	@Override
-	public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) {
-		return new DefaultInputSplitAssigner(inputSplits);
-	}
-
-	/**
-	 * Closes all resources used.
-	 */
-	@Override
-	public void close() throws IOException {
-		try {
-			if (session != null) {
-				session.close();
-			}
-		} catch (Exception e) {
-			LOG.error("Error while closing session.", e);
-		}
-
-		try {
-			if (cluster != null ) {
-				cluster.close();
-			}
-		} catch (Exception e) {
-			LOG.error("Error while closing cluster.", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java b/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java
deleted file mode 100644
index 15d8fb3..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.batch.connectors.cassandra;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
-import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import org.apache.flink.api.common.io.RichOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
-import org.apache.flink.util.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * OutputFormat to write {@link org.apache.flink.api.java.tuple.Tuple} into Apache Cassandra.
- *
- * @param <OUT> type of Tuple
- */
-public class CassandraOutputFormat<OUT extends Tuple> extends RichOutputFormat<OUT> {
-	private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormat.class);
-
-	private final String insertQuery;
-	private final ClusterBuilder builder;
-
-	private transient Cluster cluster;
-	private transient Session session;
-	private transient PreparedStatement prepared;
-	private transient FutureCallback<ResultSet> callback;
-	private transient Throwable exception = null;
-
-	public CassandraOutputFormat(String insertQuery, ClusterBuilder builder) {
-		Preconditions.checkArgument(!Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
-		Preconditions.checkArgument(builder != null, "Builder cannot be null");
-
-		this.insertQuery = insertQuery;
-		this.builder = builder;
-	}
-
-	@Override
-	public void configure(Configuration parameters) {
-		this.cluster = builder.getCluster();
-	}
-
-	/**
-	 * Opens a Session to Cassandra and initializes the prepared statement.
-	 *
-	 * @param taskNumber The number of the parallel instance.
-	 * @throws IOException Thrown, if the output could not be opened due to an
-	 *                     I/O problem.
-	 */
-	@Override
-	public void open(int taskNumber, int numTasks) throws IOException {
-		this.session = cluster.connect();
-		this.prepared = session.prepare(insertQuery);
-		this.callback = new FutureCallback<ResultSet>() {
-			@Override
-			public void onSuccess(ResultSet ignored) {
-			}
-
-			@Override
-			public void onFailure(Throwable t) {
-				exception = t;
-			}
-		};
-	}
-
-	@Override
-	public void writeRecord(OUT record) throws IOException {
-		if (exception != null) {
-			throw new IOException("write record failed", exception);
-		}
-
-		Object[] fields = new Object[record.getArity()];
-		for (int i = 0; i < record.getArity(); i++) {
-			fields[i] = record.getField(i);
-		}
-		ResultSetFuture result = session.executeAsync(prepared.bind(fields));
-		Futures.addCallback(result, callback);
-	}
-
-	/**
-	 * Closes all resources used.
-	 */
-	@Override
-	public void close() throws IOException {
-		try {
-			if (session != null) {
-				session.close();
-			}
-		} catch (Exception e) {
-			LOG.error("Error while closing session.", e);
-		}
-
-		try {
-			if (cluster != null ) {
-				cluster.close();
-			}
-		} catch (Exception e) {
-			LOG.error("Error while closing cluster.", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraCommitter.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraCommitter.java b/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraCommitter.java
deleted file mode 100644
index 63b76da..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraCommitter.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.cassandra;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.Row;
-import com.datastax.driver.core.Session;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-/**
- * CheckpointCommitter that saves information about completed checkpoints within a separate table in a cassandra
- * database.
- * 
- * <p>Entries are in the form |operator_id | subtask_id | last_completed_checkpoint|
- */
-public class CassandraCommitter extends CheckpointCommitter {
-
-	private static final long serialVersionUID = 1L;
-	
-	private final ClusterBuilder builder;
-	private transient Cluster cluster;
-	private transient Session session;
-
-	private String keySpace = "flink_auxiliary";
-	private String table = "checkpoints_";
-
-	/**
-	 * A cache of the last committed checkpoint ids per subtask index. This is used to
-	 * avoid redundant round-trips to Cassandra (see {@link #isCheckpointCommitted(int, long)}.
-	 */
-	private final Map<Integer, Long> lastCommittedCheckpoints = new HashMap<>();
-
-	public CassandraCommitter(ClusterBuilder builder) {
-		this.builder = builder;
-		ClosureCleaner.clean(builder, true);
-	}
-
-	public CassandraCommitter(ClusterBuilder builder, String keySpace) {
-		this(builder);
-		this.keySpace = keySpace;
-	}
-
-	/**
-	 * Internally used to set the job ID after instantiation.
-	 */
-	public void setJobId(String id) throws Exception {
-		super.setJobId(id);
-		table += id;
-	}
-
-	/**
-	 * Generates the necessary tables to store information.
-	 *
-	 * @throws Exception
-	 */
-	@Override
-	public void createResource() throws Exception {
-		cluster = builder.getCluster();
-		session = cluster.connect();
-
-		session.execute(String.format("CREATE KEYSPACE IF NOT EXISTS %s with replication={'class':'SimpleStrategy', 'replication_factor':1};", keySpace));
-		session.execute(String.format("CREATE TABLE IF NOT EXISTS %s.%s (sink_id text, sub_id int, checkpoint_id bigint, PRIMARY KEY (sink_id, sub_id));", keySpace, table));
-
-		try {
-			session.close();
-		} catch (Exception e) {
-			LOG.error("Error while closing session.", e);
-		}
-		try {
-			cluster.close();
-		} catch (Exception e) {
-			LOG.error("Error while closing cluster.", e);
-		}
-	}
-
-	@Override
-	public void open() throws Exception {
-		if (builder == null) {
-			throw new RuntimeException("No ClusterBuilder was set.");
-		}
-		cluster = builder.getCluster();
-		session = cluster.connect();
-	}
-
-	@Override
-	public void close() throws Exception {
-		this.lastCommittedCheckpoints.clear();
-		try {
-			session.close();
-		} catch (Exception e) {
-			LOG.error("Error while closing session.", e);
-		}
-		try {
-			cluster.close();
-		} catch (Exception e) {
-			LOG.error("Error while closing cluster.", e);
-		}
-	}
-
-	@Override
-	public void commitCheckpoint(int subtaskIdx, long checkpointId) {
-		String statement = String.format(
-			"UPDATE %s.%s set checkpoint_id=%d where sink_id='%s' and sub_id=%d;",
-			keySpace, table, checkpointId, operatorId, subtaskIdx);
-
-		session.execute(statement);
-		lastCommittedCheckpoints.put(subtaskIdx, checkpointId);
-	}
-
-	@Override
-	public boolean isCheckpointCommitted(int subtaskIdx, long checkpointId) {
-		// Pending checkpointed buffers are committed in ascending order of their
-		// checkpoint id. This way we can tell if a checkpointed buffer was committed
-		// just by asking the third-party storage system for the last checkpoint id
-		// committed by the specified subtask.
-
-		Long lastCommittedCheckpoint = lastCommittedCheckpoints.get(subtaskIdx);
-		if (lastCommittedCheckpoint == null) {
-			String statement = String.format(
-				"SELECT checkpoint_id FROM %s.%s where sink_id='%s' and sub_id=%d;",
-				keySpace, table, operatorId, subtaskIdx);
-
-			Iterator<Row> resultIt = session.execute(statement).iterator();
-			if (resultIt.hasNext()) {
-				lastCommittedCheckpoint = resultIt.next().getLong("checkpoint_id");
-				lastCommittedCheckpoints.put(subtaskIdx, lastCommittedCheckpoint);
-			}
-		}
-		return lastCommittedCheckpoint != null && checkpointId <= lastCommittedCheckpoint;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraPojoSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraPojoSink.java b/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraPojoSink.java
deleted file mode 100644
index 650c481..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraPojoSink.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.cassandra;
-
-import com.datastax.driver.mapping.Mapper;
-import com.datastax.driver.mapping.MappingManager;
-import com.google.common.util.concurrent.ListenableFuture;
-import org.apache.flink.configuration.Configuration;
-
-/**
- * Flink Sink to save data into a Cassandra cluster using 
- * <a href="http://docs.datastax.com/en/drivers/java/2.1/com/datastax/driver/mapping/Mapper.html">Mapper</a>,
- * which it uses annotations from
- * <a href="http://docs.datastax.com/en/drivers/java/2.1/com/datastax/driver/mapping/annotations/package-summary.html">
- * com.datastax.driver.mapping.annotations</a>.
- *
- * @param <IN> Type of the elements emitted by this sink
- */
-public class CassandraPojoSink<IN> extends CassandraSinkBase<IN, Void> {
-
-	private static final long serialVersionUID = 1L;
-
-	protected final Class<IN> clazz;
-	protected transient Mapper<IN> mapper;
-	protected transient MappingManager mappingManager;
-
-	/**
-	 * The main constructor for creating CassandraPojoSink
-	 *
-	 * @param clazz Class<IN> instance
-	 */
-	public CassandraPojoSink(Class<IN> clazz, ClusterBuilder builder) {
-		super(builder);
-		this.clazz = clazz;
-	}
-
-	@Override
-	public void open(Configuration configuration) {
-		super.open(configuration);
-		try {
-			this.mappingManager = new MappingManager(session);
-			this.mapper = mappingManager.mapper(clazz);
-		} catch (Exception e) {
-			throw new RuntimeException("Cannot create CassandraPojoSink with input: " + clazz.getSimpleName(), e);
-		}
-	}
-
-	@Override
-	public ListenableFuture<Void> send(IN value) {
-		return mapper.saveAsync(value);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSink.java b/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSink.java
deleted file mode 100644
index 180b638..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSink.java
+++ /dev/null
@@ -1,329 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cassandra;
-
-import com.datastax.driver.core.Cluster;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.transformations.SinkTransformation;
-import org.apache.flink.streaming.api.transformations.StreamTransformation;
-import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
-
-/**
- * This class wraps different Cassandra sink implementations to provide a common interface for all of them.
- *
- * @param <IN> input type
- */
-public class CassandraSink<IN> {
-	private final boolean useDataStreamSink;
-	private DataStreamSink<IN> sink1;
-	private SingleOutputStreamOperator<IN> sink2;
-
-	private CassandraSink(DataStreamSink<IN> sink) {
-		sink1 = sink;
-		useDataStreamSink = true;
-	}
-
-	private CassandraSink(SingleOutputStreamOperator<IN> sink) {
-		sink2 = sink;
-		useDataStreamSink = false;
-	}
-
-	private SinkTransformation<IN> getSinkTransformation() {
-		return sink1.getTransformation();
-	}
-
-	private StreamTransformation<IN> getStreamTransformation() {
-		return sink2.getTransformation();
-	}
-
-	/**
-	 * Sets the name of this sink. This name is
-	 * used by the visualization and logging during runtime.
-	 *
-	 * @return The named sink.
-	 */
-	public CassandraSink<IN> name(String name) {
-		if (useDataStreamSink) {
-			getSinkTransformation().setName(name);
-		} else {
-			getStreamTransformation().setName(name);
-		}
-		return this;
-	}
-
-	/**
-	 * Sets an ID for this operator.
-	 * <p/>
-	 * <p>The specified ID is used to assign the same operator ID across job
-	 * submissions (for example when starting a job from a savepoint).
-	 * <p/>
-	 * <p><strong>Important</strong>: this ID needs to be unique per
-	 * transformation and job. Otherwise, job submission will fail.
-	 *
-	 * @param uid The unique user-specified ID of this transformation.
-	 * @return The operator with the specified ID.
-	 */
-	public CassandraSink<IN> uid(String uid) {
-		if (useDataStreamSink) {
-			getSinkTransformation().setUid(uid);
-		} else {
-			getStreamTransformation().setUid(uid);
-		}
-		return this;
-	}
-
-	/**
-	 * Sets the parallelism for this sink. The degree must be higher than zero.
-	 *
-	 * @param parallelism The parallelism for this sink.
-	 * @return The sink with set parallelism.
-	 */
-	public CassandraSink<IN> setParallelism(int parallelism) {
-		if (useDataStreamSink) {
-			getSinkTransformation().setParallelism(parallelism);
-		} else {
-			getStreamTransformation().setParallelism(parallelism);
-		}
-		return this;
-	}
-
-	/**
-	 * Turns off chaining for this operator so thread co-location will not be
-	 * used as an optimization.
-	 * <p/>
-	 * <p/>
-	 * Chaining can be turned off for the whole
-	 * job by {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#disableOperatorChaining()}
-	 * however it is not advised for performance considerations.
-	 *
-	 * @return The sink with chaining disabled
-	 */
-	public CassandraSink<IN> disableChaining() {
-		if (useDataStreamSink) {
-			getSinkTransformation().setChainingStrategy(ChainingStrategy.NEVER);
-		} else {
-			getStreamTransformation().setChainingStrategy(ChainingStrategy.NEVER);
-		}
-		return this;
-	}
-
-	/**
-	 * Sets the slot sharing group of this operation. Parallel instances of
-	 * operations that are in the same slot sharing group will be co-located in the same
-	 * TaskManager slot, if possible.
-	 * <p/>
-	 * <p>Operations inherit the slot sharing group of input operations if all input operations
-	 * are in the same slot sharing group and no slot sharing group was explicitly specified.
-	 * <p/>
-	 * <p>Initially an operation is in the default slot sharing group. An operation can be put into
-	 * the default group explicitly by setting the slot sharing group to {@code "default"}.
-	 *
-	 * @param slotSharingGroup The slot sharing group name.
-	 */
-	public CassandraSink<IN> slotSharingGroup(String slotSharingGroup) {
-		if (useDataStreamSink) {
-			getSinkTransformation().setSlotSharingGroup(slotSharingGroup);
-		} else {
-			getStreamTransformation().setSlotSharingGroup(slotSharingGroup);
-		}
-		return this;
-	}
-
-	/**
-	 * Writes a DataStream into a Cassandra database.
-	 *
-	 * @param input input DataStream
-	 * @param <IN>  input type
-	 * @return CassandraSinkBuilder, to further configure the sink
-	 */
-	public static <IN, T extends Tuple> CassandraSinkBuilder<IN> addSink(DataStream<IN> input) {
-		if (input.getType() instanceof TupleTypeInfo) {
-			DataStream<T> tupleInput = (DataStream<T>) input;
-			return (CassandraSinkBuilder<IN>) new CassandraTupleSinkBuilder<>(tupleInput, tupleInput.getType(), tupleInput.getType().createSerializer(tupleInput.getExecutionEnvironment().getConfig()));
-		} else {
-			return new CassandraPojoSinkBuilder<>(input, input.getType(), input.getType().createSerializer(input.getExecutionEnvironment().getConfig()));
-		}
-	}
-
-	public abstract static class CassandraSinkBuilder<IN> {
-		protected final DataStream<IN> input;
-		protected final TypeSerializer<IN> serializer;
-		protected final TypeInformation<IN> typeInfo;
-		protected ClusterBuilder builder;
-		protected String query;
-		protected CheckpointCommitter committer;
-		protected boolean isWriteAheadLogEnabled;
-
-		public CassandraSinkBuilder(DataStream<IN> input, TypeInformation<IN> typeInfo, TypeSerializer<IN> serializer) {
-			this.input = input;
-			this.typeInfo = typeInfo;
-			this.serializer = serializer;
-		}
-
-		/**
-		 * Sets the query that is to be executed for every record.
-		 *
-		 * @param query query to use
-		 * @return this builder
-		 */
-		public CassandraSinkBuilder<IN> setQuery(String query) {
-			this.query = query;
-			return this;
-		}
-
-		/**
-		 * Sets the cassandra host to connect to.
-		 *
-		 * @param host host to connect to
-		 * @return this builder
-		 */
-		public CassandraSinkBuilder<IN> setHost(String host) {
-			return setHost(host, 9042);
-		}
-
-		/**
-		 * Sets the cassandra host/port to connect to.
-		 *
-		 * @param host host to connect to
-		 * @param port port to connect to
-		 * @return this builder
-		 */
-		public CassandraSinkBuilder<IN> setHost(final String host, final int port) {
-			if (this.builder != null) {
-				throw new IllegalArgumentException("Builder was already set. You must use either setHost() or setClusterBuilder().");
-			}
-			this.builder = new ClusterBuilder() {
-				@Override
-				protected Cluster buildCluster(Cluster.Builder builder) {
-					return builder.addContactPoint(host).withPort(port).build();
-				}
-			};
-			return this;
-		}
-
-		/**
-		 * Sets the ClusterBuilder for this sink. A ClusterBuilder is used to configure the connection to cassandra.
-		 *
-		 * @param builder ClusterBuilder to configure the connection to cassandra
-		 * @return this builder
-		 */
-		public CassandraSinkBuilder<IN> setClusterBuilder(ClusterBuilder builder) {
-			if (this.builder != null) {
-				throw new IllegalArgumentException("Builder was already set. You must use either setHost() or setClusterBuilder().");
-			}
-			this.builder = builder;
-			return this;
-		}
-
-		/**
-		 * Enables the write-ahead log, which allows exactly-once processing for non-deterministic algorithms that use
-		 * idempotent updates.
-		 *
-		 * @return this builder
-		 */
-		public CassandraSinkBuilder<IN> enableWriteAheadLog() {
-			this.isWriteAheadLogEnabled = true;
-			return this;
-		}
-
-		/**
-		 * Enables the write-ahead log, which allows exactly-once processing for non-deterministic algorithms that use
-		 * idempotent updates.
-		 *
-		 * @param committer CheckpointCommitter, that stores informationa bout completed checkpoints in an external
-		 *                  resource. By default this information is stored within a separate table within Cassandra.
-		 * @return this builder
-		 */
-		public CassandraSinkBuilder<IN> enableWriteAheadLog(CheckpointCommitter committer) {
-			this.isWriteAheadLogEnabled = true;
-			this.committer = committer;
-			return this;
-		}
-
-		/**
-		 * Finalizes the configuration of this sink.
-		 *
-		 * @return finalized sink
-		 * @throws Exception
-		 */
-		public abstract CassandraSink<IN> build() throws Exception;
-
-		protected void sanityCheck() {
-			if (builder == null) {
-				throw new IllegalArgumentException("Cassandra host information must be supplied using either setHost() or setClusterBuilder().");
-			}
-		}
-	}
-
-	public static class CassandraTupleSinkBuilder<IN extends Tuple> extends CassandraSinkBuilder<IN> {
-		public CassandraTupleSinkBuilder(DataStream<IN> input, TypeInformation<IN> typeInfo, TypeSerializer<IN> serializer) {
-			super(input, typeInfo, serializer);
-		}
-
-		@Override
-		protected void sanityCheck() {
-			super.sanityCheck();
-			if (query == null || query.length() == 0) {
-				throw new IllegalArgumentException("Query must not be null or empty.");
-			}
-		}
-
-		@Override
-		public CassandraSink<IN> build() throws Exception {
-			sanityCheck();
-			if (isWriteAheadLogEnabled) {
-				return committer == null
-					? new CassandraSink<>(input.transform("Cassandra Sink", null, new CassandraTupleWriteAheadSink<>(query, serializer, builder, new CassandraCommitter(builder))))
-					: new CassandraSink<>(input.transform("Cassandra Sink", null, new CassandraTupleWriteAheadSink<>(query, serializer, builder, committer)));
-			} else {
-				return new CassandraSink<>(input.addSink(new CassandraTupleSink<IN>(query, builder)).name("Cassandra Sink"));
-			}
-		}
-	}
-
-	public static class CassandraPojoSinkBuilder<IN> extends CassandraSinkBuilder<IN> {
-		public CassandraPojoSinkBuilder(DataStream<IN> input, TypeInformation<IN> typeInfo, TypeSerializer<IN> serializer) {
-			super(input, typeInfo, serializer);
-		}
-
-		@Override
-		protected void sanityCheck() {
-			super.sanityCheck();
-			if (query != null) {
-				throw new IllegalArgumentException("Specifying a query is not allowed when using a Pojo-Stream as input.");
-			}
-		}
-
-		@Override
-		public CassandraSink<IN> build() throws Exception {
-			sanityCheck();
-			if (isWriteAheadLogEnabled) {
-				throw new IllegalArgumentException("Exactly-once guarantees can only be provided for tuple types.");
-			} else {
-				return new CassandraSink<>(input.addSink(new CassandraPojoSink<>(typeInfo.getTypeClass(), builder)).name("Cassandra Sink"));
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java b/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java
deleted file mode 100644
index 49b1efa..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.cassandra;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.Session;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * CassandraSinkBase is the common abstract class of {@link CassandraPojoSink} and {@link CassandraTupleSink}.
- *
- * @param <IN> Type of the elements emitted by this sink
- */
-public abstract class CassandraSinkBase<IN, V> extends RichSinkFunction<IN> {
-	protected static final Logger LOG = LoggerFactory.getLogger(CassandraSinkBase.class);
-	protected transient Cluster cluster;
-	protected transient Session session;
-
-	protected transient Throwable exception = null;
-	protected transient FutureCallback<V> callback;
-
-	private final ClusterBuilder builder;
-
-	protected CassandraSinkBase(ClusterBuilder builder) {
-		this.builder = builder;
-		ClosureCleaner.clean(builder, true);
-	}
-
-	@Override
-	public void open(Configuration configuration) {
-		this.callback = new FutureCallback<V>() {
-			@Override
-			public void onSuccess(V ignored) {
-			}
-
-			@Override
-			public void onFailure(Throwable t) {
-				exception = t;
-				LOG.error("Error while sending value.", t);
-			}
-		};
-		this.cluster = builder.getCluster();
-		this.session = cluster.connect();
-	}
-
-	@Override
-	public void invoke(IN value) throws Exception {
-		if (exception != null) {
-			throw new IOException("invoke() failed", exception);
-		}
-		ListenableFuture<V> result = send(value);
-		Futures.addCallback(result, callback);
-	}
-
-	public abstract ListenableFuture<V> send(IN value);
-
-	@Override
-	public void close() {
-		try {
-			if (session != null) {
-				session.close();
-			}
-		} catch (Exception e) {
-			LOG.error("Error while closing session.", e);
-		}
-		try {
-			if (cluster != null) {
-				cluster.close();
-			}
-		} catch (Exception e) {
-			LOG.error("Error while closing cluster.", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleSink.java b/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleSink.java
deleted file mode 100644
index 0a9ef06..0000000
--- a/flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleSink.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.cassandra;
-
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.google.common.util.concurrent.ListenableFuture;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.configuration.Configuration;
-
-/**
- * Flink Sink to save data into a Cassandra cluster.
- *
- * @param <IN> Type of the elements emitted by this sink, it must extend {@link Tuple}
- */
-public class CassandraTupleSink<IN extends Tuple> extends CassandraSinkBase<IN, ResultSet> {
-	private final String insertQuery;
-	private transient PreparedStatement ps;
-
-	public CassandraTupleSink(String insertQuery, ClusterBuilder builder) {
-		super(builder);
-		this.insertQuery = insertQuery;
-	}
-
-	@Override
-	public void open(Configuration configuration) {
-		super.open(configuration);
-		this.ps = session.prepare(insertQuery);
-	}
-
-	@Override
-	public ListenableFuture<ResultSet> send(IN value) {
-		Object[] fields = extract(value);
-		return session.executeAsync(ps.bind(fields));
-	}
-
-	private Object[] extract(IN record) {
-		Object[] al = new Object[record.getArity()];
-		for (int i = 0; i < record.getArity(); i++) {
-			al[i] = record.getField(i);
-		}
-		return al;
-	}
-}


[11/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index d015157..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java
+++ /dev/null
@@ -1,481 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.metrics.MetricGroup;
-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 subtask's runtime context */
-	private final RuntimeContext runtimeContext;
-
-	/** 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,
-			boolean useMetrics) throws Exception
-	{
-		super(
-				sourceContext,
-				assignedPartitions,
-				watermarksPeriodic,
-				watermarksPunctuated,
-				runtimeContext.getProcessingTimeService(),
-				runtimeContext.getExecutionConfig().getAutoWatermarkInterval(),
-				runtimeContext.getUserCodeClassLoader(),
-				useMetrics);
-
-		this.deserializer = checkNotNull(deserializer);
-		this.kafkaConfig = checkNotNull(kafkaProperties);
-		this.runtimeContext = runtimeContext;
-		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.getCommittedOffsets(partitionsWithNoOffset);
-				for (KafkaTopicPartitionState<TopicAndPartition> partition : subscribedPartitions()) {
-					Long zkOffset = zkOffsets.get(partition.getKafkaTopicPartition());
-					if (zkOffset != null) {
-						// the offset in ZK represents the "next record to process", so we need to subtract it by 1
-						// to correctly represent our internally checkpointed offsets
-						partition.setOffset(zkOffset - 1);
-					}
-				}
-			}
-
-			// 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();
-			}
-
-			// register offset metrics
-			if (useMetrics) {
-				final MetricGroup kafkaMetricGroup = runtimeContext.getMetricGroup().addGroup("KafkaConsumer");
-				addOffsetStateGauge(kafkaMetricGroup);
-			}
-
-			// 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();
-			}
-
-			// clear the interruption flag
-			// this allows the joining on consumer threads (on best effort) to happen in
-			// case the initial interrupt already
-			Thread.interrupted();
-
-			// 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 (InterruptedException ignored) {
-				// waiting for the thread shutdown apparently got interrupted
-				// restore interrupted state and continue
-				Thread.currentThread().interrupt();
-			}
-			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 commitInternalOffsetsToKafka(Map<KafkaTopicPartition, Long> offsets) throws Exception {
-		ZookeeperOffsetHandler zkHandler = this.zookeeperOffsetHandler;
-		if (zkHandler != null) {
-			// the ZK handler takes care of incrementing the offsets by 1 before committing
-			zkHandler.prepareAndCommitOffsets(offsets);
-		}
-
-		// Set committed offsets in topic partition state
-		KafkaTopicPartitionState<TopicAndPartition>[] partitions = subscribedPartitions();
-		for (KafkaTopicPartitionState<TopicAndPartition> partition : partitions) {
-			Long offset = offsets.get(partition.getKafkaTopicPartition());
-			if (offset != null) {
-				partition.setCommittedOffset(offset);
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  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, runtimeContext.getUserCodeClassLoader());
-
-		// 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)",
-				runtimeContext.getTaskName(), 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/de4fe3b7/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
deleted file mode 100644
index 4d61e53..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.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/de4fe3b7/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
deleted file mode 100644
index d8d927d..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index 27d90f2..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.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> offsetsToCommit = new HashMap<>(partitionStates.length);
-				for (KafkaTopicPartitionState<?> partitionState : partitionStates) {
-					offsetsToCommit.put(partitionState.getKafkaTopicPartition(), partitionState.getOffset());
-				}
-				
-				offsetHandler.prepareAndCommitOffsets(offsetsToCommit);
-			}
-		}
-		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/de4fe3b7/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
deleted file mode 100644
index 35e491a..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java
+++ /dev/null
@@ -1,504 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.PropertiesUtil.getInt;
-
-/**
- * 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 = getInt(config, "socket.timeout.ms", 30000);
-		this.minBytes = getInt(config, "fetch.min.bytes", 1);
-		this.maxWait = getInt(config, "fetch.wait.max.ms", 100);
-		this.fetchSize = getInt(config, "fetch.message.max.bytes", 1048576);
-		this.bufferSize = getInt(config, "socket.receive.buffer.bytes", 65536);
-		this.reconnectLimit = getInt(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/de4fe3b7/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
deleted file mode 100644
index 8f2ef09..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import kafka.utils.ZKGroupTopicDirs;
-
-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.kafka.clients.consumer.ConsumerConfig;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-/**
- * Handler for committing Kafka offsets to Zookeeper and to retrieve them again.
- */
-public class ZookeeperOffsetHandler {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ZookeeperOffsetHandler.class);
-
-	private final String groupId;
-
-	private final CuratorFramework curatorClient;
-
-
-	public ZookeeperOffsetHandler(Properties props) {
-		this.groupId = props.getProperty(ConsumerConfig.GROUP_ID_CONFIG);
-		if (this.groupId == null) {
-			throw new IllegalArgumentException("Required property '"
-					+ ConsumerConfig.GROUP_ID_CONFIG + "' has not been set");
-		}
-		
-		String zkConnect = props.getProperty("zookeeper.connect");
-		if (zkConnect == null) {
-			throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set");
-		}
-
-		// we use Curator's default timeouts
-		int sessionTimeoutMs =  Integer.valueOf(props.getProperty("zookeeper.session.timeout.ms", "60000"));
-		int connectionTimeoutMs = Integer.valueOf(props.getProperty("zookeeper.connection.timeout.ms", "15000"));
-		
-		// undocumented config options allowing users to configure the retry policy. (they are "flink." prefixed as they are no official kafka configs)
-		int backoffBaseSleepTime = Integer.valueOf(props.getProperty("flink.zookeeper.base-sleep-time.ms", "100"));
-		int backoffMaxRetries =  Integer.valueOf(props.getProperty("flink.zookeeper.max-retries", "10"));
-		
-		RetryPolicy retryPolicy = new ExponentialBackoffRetry(backoffBaseSleepTime, backoffMaxRetries);
-		curatorClient = CuratorFrameworkFactory.newClient(zkConnect, sessionTimeoutMs, connectionTimeoutMs, retryPolicy);
-		curatorClient.start();
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Offset access and manipulation
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Commits offsets for Kafka partitions to ZooKeeper. The given offsets to this method should be the offsets of
-	 * the last processed records; this method will take care of incrementing the offsets by 1 before committing them so
-	 * that the committed offsets to Zookeeper represent the next record to process.
-	 * 
-	 * @param internalOffsets The internal offsets (representing last processed records) for the partitions to commit.
-	 * @throws Exception The method forwards exceptions.
-	 */
-	public void prepareAndCommitOffsets(Map<KafkaTopicPartition, Long> internalOffsets) throws Exception {
-		for (Map.Entry<KafkaTopicPartition, Long> entry : internalOffsets.entrySet()) {
-			KafkaTopicPartition tp = entry.getKey();
-
-			Long lastProcessedOffset = entry.getValue();
-			if (lastProcessedOffset != null && lastProcessedOffset >= 0) {
-				setOffsetInZooKeeper(curatorClient, groupId, tp.getTopic(), tp.getPartition(), lastProcessedOffset + 1);
-			}
-		}
-	}
-
-	/**
-	 * @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> getCommittedOffsets(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());
-
-			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);
-			}
-		}
-		return ret;
-	}
-
-	/**
-	 * Closes the offset handler.
-	 * 
-	 * @throws IOException Thrown, if the handler cannot be closed properly.
-	 */
-	public void close() throws IOException {
-		curatorClient.close();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Communication with Zookeeper
-	// ------------------------------------------------------------------------
-	
-	public static void setOffsetInZooKeeper(CuratorFramework curatorClient, String groupId, String topic, int partition, long offset) throws Exception {
-		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, topic);
-		String path = topicDirs.consumerOffsetDir() + "/" + partition;
-		curatorClient.newNamespaceAwareEnsurePath(path).ensure(curatorClient.getZookeeperClient());
-		byte[] data = Long.toString(offset).getBytes();
-		curatorClient.setData().forPath(path, data);
-	}
-
-	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());
-		
-		byte[] data = curatorClient.getData().forPath(path);
-		
-		if (data == null) {
-			return null;
-		} else {
-			String asString = new String(data);
-			if (asString.length() == 0) {
-				return null;
-			} else {
-				try {
-					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/de4fe3b7/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
deleted file mode 100644
index fabb0fe..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.curator.framework.CuratorFramework;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.runtime.client.JobCancellationException;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
-import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
-import org.apache.flink.streaming.connectors.kafka.testutils.JobManagerCommunicationUtils;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Properties;
-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 {
-
-	// ------------------------------------------------------------------------
-	//  Suite of Tests
-	// ------------------------------------------------------------------------
-
-	@Test(timeout = 60000)
-	public void testFailOnNoBroker() throws Exception {
-		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();
-	}
-
-	// --- canceling / failures ---
-
-	@Test(timeout = 60000)
-	public void testCancelingEmptyTopic() throws Exception {
-		runCancelingOnEmptyInputTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testCancelingFullTopic() throws Exception {
-		runCancelingOnFullInputTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testFailOnDeploy() throws Exception {
-		runFailOnDeployTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testInvalidOffset() throws Exception {
-		
-		final int parallelism = 1;
-		
-		// write 20 messages into topic:
-		final String topic = writeSequence("invalidOffsetTopic", 20, parallelism, 1);
-
-		// set invalid offset:
-		CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
-		ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardProps.getProperty("group.id"), topic, 0, 1234);
-		curatorClient.close();
-
-		// 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);
-	}
-
-	// --- source to partition mappings and exactly once ---
-
-	@Test(timeout = 60000)
-	public void testOneToOneSources() throws Exception {
-		runOneToOneExactlyOnceTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testOneSourceMultiplePartitions() throws Exception {
-		runOneSourceMultiplePartitionsExactlyOnceTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testMultipleSourcesOnePartition() throws Exception {
-		runMultipleSourcesOnePartitionExactlyOnceTest();
-	}
-
-	// --- broker failure ---
-
-	@Test(timeout = 60000)
-	public void testBrokerFailure() throws Exception {
-		runBrokerFailureTest();
-	}
-
-	// --- offset committing ---
-
-	@Test(timeout = 60000)
-	public void testCommitOffsetsToZookeeper() throws Exception {
-		runCommitOffsetsToKafka();
-	}
-
-	@Test(timeout = 60000)
-	public void testStartFromZookeeperCommitOffsets() throws Exception {
-		runStartFromKafkaCommitOffsets();
-	}
-
-	@Test(timeout = 60000)
-	public void testAutoOffsetRetrievalAndCommitToZookeeper() throws Exception {
-		runAutoOffsetRetrievalAndCommitToKafka();
-	}
-
-	@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());
-		}
-	}
-
-	@Test(timeout = 60000)
-	public void testOffsetAutocommitTest() throws Exception {
-		final int parallelism = 3;
-
-		// write a sequence from 0 to 99 to each of the 3 partitions.
-		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
-		// sure that we commit roughly 3-4 times while reading, however
-		// at least once.
-		Properties readProps = new Properties();
-		readProps.putAll(standardProps);
-		readProps.setProperty("auto.commit.interval.ms", "500");
-
-		// read so that the offset can be committed to ZK
-		readSequence(env, readProps, parallelism, topicName, 100, 0);
-
-		// 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);
-		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 != 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);
-	}
-
-	// --- special executions ---
-
-	@Test(timeout = 60000)
-	public void testBigRecordJob() throws Exception {
-		runBigRecordTestTopology();
-	}
-
-	@Test(timeout = 60000)
-	public void testMultipleTopics() throws Exception {
-		runProduceConsumeMultipleTopics();
-	}
-
-	@Test(timeout = 60000)
-	public void testAllDeletes() throws Exception {
-		runAllDeletesTest();
-	}
-
-	@Test(timeout=60000)
-	public void testEndOfStream() throws Exception {
-		runEndOfStreamTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testMetrics() throws Throwable {
-		runMetricsTest();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
deleted file mode 100644
index 6d0b140..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-import java.util.Properties;
-
-public class Kafka08JsonTableSinkTest extends KafkaTableSinkTestBase {
-
-	@Override
-	protected KafkaTableSink createTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner,
-			final FlinkKafkaProducerBase<Row> kafkaProducer) {
-
-		return new Kafka08JsonTableSink(topic, properties, partitioner) {
-			@Override
-			protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties,
-					SerializationSchema<Row> serializationSchema, KafkaPartitioner<Row> partitioner) {
-				return kafkaProducer;
-			}
-		};
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	protected SerializationSchema<Row> getSerializationSchema() {
-		return new JsonRowSerializationSchema(FIELD_NAMES);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
deleted file mode 100644
index a2d66ac..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import java.util.Properties;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
-
-public class Kafka08JsonTableSourceTest extends KafkaTableSourceTestBase {
-
-	@Override
-	protected KafkaTableSource createTableSource(String topic, Properties properties, String[] fieldNames, TypeInformation<?>[] typeInfo) {
-		return new Kafka08JsonTableSource(topic, properties, fieldNames, typeInfo);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	protected Class<DeserializationSchema<Row>> getDeserializationSchema() {
-		return (Class) JsonRowDeserializationSchema.class;
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	protected Class<FlinkKafkaConsumerBase<Row>> getFlinkKafkaConsumer() {
-		return (Class) FlinkKafkaConsumer08.class;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java
deleted file mode 100644
index 5c951db..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-
-import org.junit.Test;
-
-@SuppressWarnings("serial")
-public class Kafka08ProducerITCase extends KafkaProducerTestBase {
-
-	@Test
-	public void testCustomPartitioning() {
-		runCustomPartitioningTest();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index 9520f55..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.Collections;
-import java.util.Properties;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.junit.Test;
-
-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");
-
-			FlinkKafkaConsumer08<String> consumer = new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"), new SimpleStringSchema(), props);
-			consumer.open(new Configuration());
-			fail();
-		}
-		catch (Exception e) {
-			assertTrue(e.getMessage().contains("Unable to retrieve any partitions"));
-		}
-	}
-
-	@Test
-	public void testAllBoostrapServerHostsAreInvalid() {
-		try {
-			String zookeeperConnect = "localhost:56794";
-			String bootstrapServers = "indexistentHost:11111";
-			String groupId = "non-existent-group";
-			Properties props = createKafkaProps(zookeeperConnect, bootstrapServers, groupId);
-			FlinkKafkaConsumer08<String> consumer = new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"),
-					new SimpleStringSchema(), props);
-			consumer.open(new Configuration());
-			fail();
-		} catch (Exception e) {
-			assertTrue("Exception should be thrown containing 'all bootstrap servers invalid' message!",
-					e.getMessage().contains("All the servers provided in: '" + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG
-							+ "' config are invalid"));
-		}
-	}
-
-	@Test
-	public void testAtLeastOneBootstrapServerHostIsValid() {
-		try {
-			String zookeeperConnect = "localhost:56794";
-			// we declare one valid boostrap server, namely the one with
-			// 'localhost'
-			String bootstrapServers = "indexistentHost:11111, localhost:22222";
-			String groupId = "non-existent-group";
-			Properties props = createKafkaProps(zookeeperConnect, bootstrapServers, groupId);
-			FlinkKafkaConsumer08<String> consumer = new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"),
-					new SimpleStringSchema(), props);
-			consumer.open(new Configuration());
-			fail();
-		} catch (Exception e) {
-			// test is not failing because we have one valid boostrap server
-			assertTrue("The cause of the exception should not be 'all boostrap server are invalid'!",
-					!e.getMessage().contains("All the hosts provided in: " + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG
-							+ " config are invalid"));
-		}
-	}
-	
-	private Properties createKafkaProps(String zookeeperConnect, String bootstrapServers, String groupId) {
-		Properties props = new Properties();
-		props.setProperty("zookeeper.connect", zookeeperConnect);
-		props.setProperty("bootstrap.servers", bootstrapServers);
-		props.setProperty("group.id", groupId);
-		return props;
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
deleted file mode 100644
index 91fc286..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
-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.Collections;
-import java.util.concurrent.Future;
-
-
-import static org.mockito.Mockito.*;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
-import static org.junit.Assert.*;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(FlinkKafkaProducerBase.class)
-public class KafkaProducerTest extends TestLogger {
-	
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testPropagateExceptions() {
-		try {
-			// mock kafka producer
-			KafkaProducer<?, ?> kafkaProducerMock = mock(KafkaProducer.class);
-			
-			// partition setup
-			when(kafkaProducerMock.partitionsFor(anyString())).thenReturn(
-				// returning a unmodifiable list to mimic KafkaProducer#partitionsFor() behaviour
-				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)))
-				.thenAnswer(new Answer<Future<RecordMetadata>>() {
-					@Override
-					public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
-						Callback callback = (Callback) invocation.getArguments()[1];
-						callback.onCompletion(null, new Exception("Test error"));
-						return null;
-					}
-				});
-			
-			// make sure the FlinkKafkaProducer instantiates our mock producer
-			whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock);
-			
-			// (1) producer that propagates errors
-
-			FlinkKafkaProducer08<String> producerPropagating = new FlinkKafkaProducer08<>(
-					"mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), null);
-
-			OneInputStreamOperatorTestHarness<String, Object> testHarness =
-					new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producerPropagating));
-
-			testHarness.open();
-
-			try {
-				testHarness.processElement(new StreamRecord<>("value"));
-				testHarness.processElement(new StreamRecord<>("value"));
-				fail("This should fail with an exception");
-			}
-			catch (Exception e) {
-				assertNotNull(e.getCause());
-				assertNotNull(e.getCause().getMessage());
-				assertTrue(e.getCause().getMessage().contains("Test error"));
-			}
-
-			testHarness.close();
-
-			// (2) producer that only logs errors
-
-			FlinkKafkaProducer08<String> producerLogging = new FlinkKafkaProducer08<>(
-					"mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), null);
-			producerLogging.setLogFailuresOnly(true);
-
-			testHarness = new OneInputStreamOperatorTestHarness<>(new StreamSink(producerLogging));
-
-			testHarness.open();
-
-			testHarness.processElement(new StreamRecord<>("value"));
-			testHarness.processElement(new StreamRecord<>("value"));
-
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}


[48/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java
deleted file mode 100644
index c32f5da..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import org.apache.hadoop.io.Writable;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-public class StringArrayWritable implements Writable, Comparable<StringArrayWritable> {
-	
-	private String[] array = new String[0];
-	
-	public StringArrayWritable() {
-		super();
-	}
-	
-	public StringArrayWritable(String[] array) {
-		this.array = array;
-	}
-	
-	@Override
-	public void write(DataOutput out) throws IOException {
-		out.writeInt(this.array.length);
-		
-		for(String str : this.array) {
-			byte[] b = str.getBytes();
-			out.writeInt(b.length);
-			out.write(b);
-		}
-	}
-	
-	@Override
-	public void readFields(DataInput in) throws IOException {
-		this.array = new String[in.readInt()];
-		
-		for(int i = 0; i < this.array.length; i++) {
-			byte[] b = new byte[in.readInt()];
-			in.readFully(b);
-			this.array[i] = new String(b);
-		}
-	}
-	
-	@Override
-	public int compareTo(StringArrayWritable o) {
-		if(this.array.length != o.array.length) {
-			return this.array.length - o.array.length;
-		}
-		
-		for(int i = 0; i < this.array.length; i++) {
-			int comp = this.array[i].compareTo(o.array[i]);
-			if(comp != 0) {
-				return comp;
-			}
-		}
-		return 0;
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-		if(!(obj instanceof StringArrayWritable)) {
-			return false;
-		}
-		return this.compareTo((StringArrayWritable) obj) == 0;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java
deleted file mode 100644
index 96f844c..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import org.apache.flink.api.common.typeutils.ComparatorTestBase;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-public class WritableComparatorTest extends ComparatorTestBase<StringArrayWritable> {
-	
-	StringArrayWritable[] data = new StringArrayWritable[]{
-			new StringArrayWritable(new String[]{}),
-			new StringArrayWritable(new String[]{""}),
-			new StringArrayWritable(new String[]{"a","a"}),
-			new StringArrayWritable(new String[]{"a","b"}),
-			new StringArrayWritable(new String[]{"c","c"}),
-			new StringArrayWritable(new String[]{"d","f"}),
-			new StringArrayWritable(new String[]{"d","m"}),
-			new StringArrayWritable(new String[]{"z","x"}),
-			new StringArrayWritable(new String[]{"a","a", "a"})
-	};
-	
-	@Override
-	protected TypeComparator<StringArrayWritable> createComparator(boolean ascending) {
-		return new WritableComparator<StringArrayWritable>(ascending, StringArrayWritable.class);
-	}
-	
-	@Override
-	protected TypeSerializer<StringArrayWritable> createSerializer() {
-		return new WritableSerializer<StringArrayWritable>(StringArrayWritable.class);
-	}
-	
-	@Override
-	protected StringArrayWritable[] getSortedTestData() {
-		return data;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java
deleted file mode 100644
index 94e759d..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import org.apache.flink.api.common.typeutils.ComparatorTestBase;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-import java.util.UUID;
-
-public class WritableComparatorUUIDTest extends ComparatorTestBase<WritableID> {
-	@Override
-	protected TypeComparator<WritableID> createComparator(boolean ascending) {
-		return new WritableComparator<>(ascending, WritableID.class);
-	}
-
-	@Override
-	protected TypeSerializer<WritableID> createSerializer() {
-		return new WritableSerializer<>(WritableID.class);
-	}
-
-	@Override
-	protected WritableID[] getSortedTestData() {
-		return new WritableID[] {
-			new WritableID(new UUID(0, 0)),
-			new WritableID(new UUID(1, 0)),
-			new WritableID(new UUID(1, 1))
-		};
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java
deleted file mode 100644
index 4274cf6..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import org.apache.hadoop.io.WritableComparable;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.UUID;
-
-public class WritableID implements WritableComparable<WritableID> {
-	private UUID uuid;
-
-	public WritableID() {
-		this.uuid = UUID.randomUUID();
-	}
-
-	public WritableID(UUID uuid) {
-		this.uuid = uuid;
-	}
-
-	@Override
-	public int compareTo(WritableID o) {
-		return this.uuid.compareTo(o.uuid);
-	}
-
-	@Override
-	public void write(DataOutput dataOutput) throws IOException {
-		dataOutput.writeLong(uuid.getMostSignificantBits());
-		dataOutput.writeLong(uuid.getLeastSignificantBits());
-	}
-
-	@Override
-	public void readFields(DataInput dataInput) throws IOException {
-		this.uuid = new UUID(dataInput.readLong(), dataInput.readLong());
-	}
-
-	@Override
-	public String toString() {
-		return uuid.toString();
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		WritableID id = (WritableID) o;
-
-		return !(uuid != null ? !uuid.equals(id.uuid) : id.uuid != null);
-	}
-
-	@Override
-	public int hashCode() {
-		return uuid != null ? uuid.hashCode() : 0;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java
deleted file mode 100644
index bb5f4d4..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.SerializerTestInstance;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.WritableTypeInfo;
-import org.junit.Test;
-
-public class WritableSerializerTest {
-	
-	@Test
-	public void testStringArrayWritable() {
-		StringArrayWritable[] data = new StringArrayWritable[]{
-				new StringArrayWritable(new String[]{}),
-				new StringArrayWritable(new String[]{""}),
-				new StringArrayWritable(new String[]{"a","a"}),
-				new StringArrayWritable(new String[]{"a","b"}),
-				new StringArrayWritable(new String[]{"c","c"}),
-				new StringArrayWritable(new String[]{"d","f"}),
-				new StringArrayWritable(new String[]{"d","m"}),
-				new StringArrayWritable(new String[]{"z","x"}),
-				new StringArrayWritable(new String[]{"a","a", "a"})
-		};
-		
-		WritableTypeInfo<StringArrayWritable> writableTypeInfo = (WritableTypeInfo<StringArrayWritable>) TypeExtractor.getForObject(data[0]);
-		WritableSerializer<StringArrayWritable> writableSerializer = (WritableSerializer<StringArrayWritable>) writableTypeInfo.createSerializer(new ExecutionConfig());
-		
-		SerializerTestInstance<StringArrayWritable> testInstance = new SerializerTestInstance<StringArrayWritable>(writableSerializer,writableTypeInfo.getTypeClass(), -1, data);
-		
-		testInstance.testAll();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java
deleted file mode 100644
index 2af7730..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import org.apache.flink.api.common.typeutils.SerializerTestBase;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-import java.util.UUID;
-
-public class WritableSerializerUUIDTest extends SerializerTestBase<WritableID> {
-	@Override
-	protected TypeSerializer<WritableID> createSerializer() {
-		return new WritableSerializer<>(WritableID.class);
-	}
-
-	@Override
-	protected int getLength() {
-		return -1;
-	}
-
-	@Override
-	protected Class<WritableID> getTypeClass() {
-		return WritableID.class;
-	}
-
-	@Override
-	protected WritableID[] getTestData() {
-		return new WritableID[] {
-			new WritableID(new UUID(0, 0)),
-			new WritableID(new UUID(1, 0)),
-			new WritableID(new UUID(1, 1))
-		};
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java
deleted file mode 100644
index 6f7673b..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hadoopcompatibility;
-
-import org.apache.flink.api.java.utils.AbstractParameterToolTest;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.junit.Test;
-
-import java.io.IOException;
-
-public class HadoopUtilsTest extends AbstractParameterToolTest {
-
-	@Test
-	public void testParamsFromGenericOptionsParser() throws IOException {
-		ParameterTool parameter = HadoopUtils.paramsFromGenericOptionsParser(new String[]{"-D", "input=myInput", "-DexpectedCount=15"});
-		validate(parameter);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
deleted file mode 100644
index 4d1acb4..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.hadoopcompatibility.mapred;
-
-import java.io.IOException;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopMapFunction;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class HadoopMapFunctionITCase extends MultipleProgramsTestBase {
-
-	public HadoopMapFunctionITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Test
-	public void testNonPassingMapper() throws Exception{
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env);
-		DataSet<Tuple2<IntWritable, Text>> nonPassingFlatMapDs = ds.
-				flatMap(new HadoopMapFunction<IntWritable, Text, IntWritable, Text>(new NonPassingMapper()));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		nonPassingFlatMapDs.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE);
-		env.execute();
-
-		compareResultsByLinesInMemory("\n", resultPath);
-	}
-
-	@Test
-	public void testDataDuplicatingMapper() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env);
-		DataSet<Tuple2<IntWritable, Text>> duplicatingFlatMapDs = ds.
-				flatMap(new HadoopMapFunction<IntWritable, Text, IntWritable, Text>(new DuplicatingMapper()));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		duplicatingFlatMapDs.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE);
-		env.execute();
-
-		String expected = "(1,Hi)\n" + "(1,HI)\n" +
-				"(2,Hello)\n" + "(2,HELLO)\n" +
-				"(3,Hello world)\n" + "(3,HELLO WORLD)\n" +
-				"(4,Hello world, how are you?)\n" + "(4,HELLO WORLD, HOW ARE YOU?)\n" +
-				"(5,I am fine.)\n" + "(5,I AM FINE.)\n" +
-				"(6,Luke Skywalker)\n" + "(6,LUKE SKYWALKER)\n" +
-				"(7,Comment#1)\n" + "(7,COMMENT#1)\n" +
-				"(8,Comment#2)\n" + "(8,COMMENT#2)\n" +
-				"(9,Comment#3)\n" + "(9,COMMENT#3)\n" +
-				"(10,Comment#4)\n" + "(10,COMMENT#4)\n" +
-				"(11,Comment#5)\n" + "(11,COMMENT#5)\n" +
-				"(12,Comment#6)\n" + "(12,COMMENT#6)\n" +
-				"(13,Comment#7)\n" + "(13,COMMENT#7)\n" +
-				"(14,Comment#8)\n" + "(14,COMMENT#8)\n" +
-				"(15,Comment#9)\n" + "(15,COMMENT#9)\n" +
-				"(16,Comment#10)\n" + "(16,COMMENT#10)\n" +
-				"(17,Comment#11)\n" + "(17,COMMENT#11)\n" +
-				"(18,Comment#12)\n" + "(18,COMMENT#12)\n" +
-				"(19,Comment#13)\n" + "(19,COMMENT#13)\n" +
-				"(20,Comment#14)\n" + "(20,COMMENT#14)\n" +
-				"(21,Comment#15)\n" + "(21,COMMENT#15)\n";
-
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-
-	@Test
-	public void testConfigurableMapper() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		JobConf conf = new JobConf();
-		conf.set("my.filterPrefix", "Hello");
-
-		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env);
-		DataSet<Tuple2<IntWritable, Text>> hellos = ds.
-				flatMap(new HadoopMapFunction<IntWritable, Text, IntWritable, Text>(new ConfigurableMapper(), conf));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		hellos.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE);
-		env.execute();
-
-		String expected = "(2,Hello)\n" +
-				"(3,Hello world)\n" +
-				"(4,Hello world, how are you?)\n";
-
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-	
-
-	
-	public static class NonPassingMapper implements Mapper<IntWritable, Text, IntWritable, Text> {
-		
-		@Override
-		public void map(final IntWritable k, final Text v, 
-				final OutputCollector<IntWritable, Text> out, final Reporter r) throws IOException {
-			if ( v.toString().contains("bananas") ) {
-				out.collect(k,v);
-			}
-		}
-		
-		@Override
-		public void configure(final JobConf arg0) { }
-
-		@Override
-		public void close() throws IOException { }
-	}
-	
-	public static class DuplicatingMapper implements Mapper<IntWritable, Text, IntWritable, Text> {
-		
-		@Override
-		public void map(final IntWritable k, final Text v, 
-				final OutputCollector<IntWritable, Text> out, final Reporter r) throws IOException {
-			out.collect(k, v);
-			out.collect(k, new Text(v.toString().toUpperCase()));
-		}
-		
-		@Override
-		public void configure(final JobConf arg0) { }
-
-		@Override
-		public void close() throws IOException { }
-	}
-	
-	public static class ConfigurableMapper implements Mapper<IntWritable, Text, IntWritable, Text> {
-		private String filterPrefix;
-		
-		@Override
-		public void map(IntWritable k, Text v, OutputCollector<IntWritable, Text> out, Reporter r)
-				throws IOException {
-			if(v.toString().startsWith(filterPrefix)) {
-				out.collect(k, v);
-			}
-		}
-		
-		@Override
-		public void configure(JobConf c) {
-			filterPrefix = c.get("my.filterPrefix");
-		}
-
-		@Override
-		public void close() throws IOException { }
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
deleted file mode 100644
index 13d971c..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.hadoopcompatibility.mapred;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceCombineFunction;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceFunction;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.hamcrest.core.IsEqual;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class HadoopReduceCombineFunctionITCase extends MultipleProgramsTestBase {
-
-	public HadoopReduceCombineFunctionITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Test
-	public void testStandardCountingWithCombiner() throws Exception{
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<IntWritable, IntWritable>> ds = HadoopTestData.getKVPairDataSet(env).
-				map(new Mapper1());
-
-		DataSet<Tuple2<IntWritable, IntWritable>> counts = ds.
-				groupBy(0).
-				reduceGroup(new HadoopReduceCombineFunction<IntWritable, IntWritable, IntWritable, IntWritable>(
-						new SumReducer(), new SumReducer()));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		counts.writeAsText(resultPath);
-		env.execute();
-
-		String expected = "(0,5)\n"+
-				"(1,6)\n" +
-				"(2,6)\n" +
-				"(3,4)\n";
-
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-
-	@Test
-	public void testUngroupedHadoopReducer() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<IntWritable, IntWritable>> ds = HadoopTestData.getKVPairDataSet(env).
-				map(new Mapper2());
-
-		DataSet<Tuple2<IntWritable, IntWritable>> sum = ds.
-				reduceGroup(new HadoopReduceCombineFunction<IntWritable, IntWritable, IntWritable, IntWritable>(
-						new SumReducer(), new SumReducer()));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		sum.writeAsText(resultPath);
-		env.execute();
-
-		String expected = "(0,231)\n";
-
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-
-	@Test
-	public void testCombiner() throws Exception {
-		org.junit.Assume.assumeThat(mode, new IsEqual<TestExecutionMode>(TestExecutionMode.CLUSTER));
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<IntWritable, IntWritable>> ds = HadoopTestData.getKVPairDataSet(env).
-				map(new Mapper3());
-
-		DataSet<Tuple2<IntWritable, IntWritable>> counts = ds.
-				groupBy(0).
-				reduceGroup(new HadoopReduceCombineFunction<IntWritable, IntWritable, IntWritable, IntWritable>(
-						new SumReducer(), new KeyChangingReducer()));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		counts.writeAsText(resultPath);
-		env.execute();
-
-		String expected = "(0,5)\n"+
-				"(1,6)\n" +
-				"(2,5)\n" +
-				"(3,5)\n";
-
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-
-	@Test
-	public void testConfigurationViaJobConf() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		JobConf conf = new JobConf();
-		conf.set("my.cntPrefix", "Hello");
-
-		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env).
-				map(new Mapper4());
-
-		DataSet<Tuple2<IntWritable, IntWritable>> hellos = ds.
-				groupBy(0).
-				reduceGroup(new HadoopReduceFunction<IntWritable, Text, IntWritable, IntWritable>(
-						new ConfigurableCntReducer(), conf));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		hellos.writeAsText(resultPath);
-		env.execute();
-
-		// return expected result
-		String expected = "(0,0)\n"+
-				"(1,0)\n" +
-				"(2,1)\n" +
-				"(3,1)\n" +
-				"(4,1)\n";
-
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-	
-	public static class SumReducer implements Reducer<IntWritable, IntWritable, IntWritable, IntWritable> {
-
-		@Override
-		public void reduce(IntWritable k, Iterator<IntWritable> v, OutputCollector<IntWritable, IntWritable> out, Reporter r)
-				throws IOException {
-			
-			int sum = 0;
-			while(v.hasNext()) {
-				sum += v.next().get();
-			}
-			out.collect(k, new IntWritable(sum));
-		}
-		
-		@Override
-		public void configure(JobConf arg0) { }
-
-		@Override
-		public void close() throws IOException { }
-	}
-	
-	public static class KeyChangingReducer implements Reducer<IntWritable, IntWritable, IntWritable, IntWritable> {
-
-		@Override
-		public void reduce(IntWritable k, Iterator<IntWritable> v, OutputCollector<IntWritable, IntWritable> out, Reporter r)
-				throws IOException {
-			while(v.hasNext()) {
-				out.collect(new IntWritable(k.get() % 4), v.next());
-			}
-		}
-		
-		@Override
-		public void configure(JobConf arg0) { }
-
-		@Override
-		public void close() throws IOException { }
-	}
-	
-	public static class ConfigurableCntReducer implements Reducer<IntWritable, Text, IntWritable, IntWritable> {
-		private String countPrefix;
-		
-		@Override
-		public void reduce(IntWritable k, Iterator<Text> vs, OutputCollector<IntWritable, IntWritable> out, Reporter r)
-				throws IOException {
-			int commentCnt = 0;
-			while(vs.hasNext()) {
-				String v = vs.next().toString();
-				if(v.startsWith(this.countPrefix)) {
-					commentCnt++;
-				}
-			}
-			out.collect(k, new IntWritable(commentCnt));
-		}
-		
-		@Override
-		public void configure(final JobConf c) { 
-			this.countPrefix = c.get("my.cntPrefix");
-		}
-
-		@Override
-		public void close() throws IOException { }
-	}
-
-	public static class Mapper1 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable,
-			IntWritable>> {
-		private static final long serialVersionUID = 1L;
-		Tuple2<IntWritable,IntWritable> outT = new Tuple2<IntWritable,IntWritable>();
-		@Override
-		public Tuple2<IntWritable, IntWritable> map(Tuple2<IntWritable, Text> v)
-		throws Exception {
-			outT.f0 = new IntWritable(v.f0.get() / 6);
-			outT.f1 = new IntWritable(1);
-			return outT;
-		}
-	}
-
-	public static class Mapper2 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable,
-			IntWritable>> {
-		private static final long serialVersionUID = 1L;
-		Tuple2<IntWritable,IntWritable> outT = new Tuple2<IntWritable,IntWritable>();
-		@Override
-		public Tuple2<IntWritable, IntWritable> map(Tuple2<IntWritable, Text> v)
-		throws Exception {
-			outT.f0 = new IntWritable(0);
-			outT.f1 = v.f0;
-			return outT;
-		}
-	}
-
-	public static class Mapper3 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable, IntWritable>> {
-		private static final long serialVersionUID = 1L;
-		Tuple2<IntWritable,IntWritable> outT = new Tuple2<IntWritable,IntWritable>();
-		@Override
-		public Tuple2<IntWritable, IntWritable> map(Tuple2<IntWritable, Text> v)
-		throws Exception {
-			outT.f0 = v.f0;
-			outT.f1 = new IntWritable(1);
-			return outT;
-		}
-	}
-
-	public static class Mapper4 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable, Text>> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Tuple2<IntWritable, Text> map(Tuple2<IntWritable, Text> v)
-		throws Exception {
-			v.f0 = new IntWritable(v.f0.get() % 5);
-			return v;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
deleted file mode 100644
index abc0e9c..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.hadoopcompatibility.mapred;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceFunction;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class HadoopReduceFunctionITCase extends MultipleProgramsTestBase {
-
-	public HadoopReduceFunctionITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Test
-	public void testStandardGrouping() throws Exception{
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env).
-				map(new Mapper1());
-
-		DataSet<Tuple2<IntWritable, IntWritable>> commentCnts = ds.
-				groupBy(0).
-				reduceGroup(new HadoopReduceFunction<IntWritable, Text, IntWritable, IntWritable>(new CommentCntReducer()));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		commentCnts.writeAsText(resultPath);
-		env.execute();
-
-		String expected = "(0,0)\n"+
-				"(1,3)\n" +
-				"(2,5)\n" +
-				"(3,5)\n" +
-				"(4,2)\n";
-
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-
-	@Test
-	public void testUngroupedHadoopReducer() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env);
-
-		DataSet<Tuple2<IntWritable, IntWritable>> commentCnts = ds.
-				reduceGroup(new HadoopReduceFunction<IntWritable, Text, IntWritable, IntWritable>(new AllCommentCntReducer()));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		commentCnts.writeAsText(resultPath);
-		env.execute();
-
-		String expected = "(42,15)\n";
-
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-
-	@Test
-	public void testConfigurationViaJobConf() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		JobConf conf = new JobConf();
-		conf.set("my.cntPrefix", "Hello");
-
-		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env).
-				map(new Mapper2());
-
-		DataSet<Tuple2<IntWritable, IntWritable>> helloCnts = ds.
-				groupBy(0).
-				reduceGroup(new HadoopReduceFunction<IntWritable, Text, IntWritable, IntWritable>(
-						new ConfigurableCntReducer(), conf));
-
-		String resultPath = tempFolder.newFile().toURI().toString();
-
-		helloCnts.writeAsText(resultPath);
-		env.execute();
-
-		String expected = "(0,0)\n"+
-				"(1,0)\n" +
-				"(2,1)\n" +
-				"(3,1)\n" +
-				"(4,1)\n";
-
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-	
-	public static class CommentCntReducer implements Reducer<IntWritable, Text, IntWritable, IntWritable> {
-		
-		@Override
-		public void reduce(IntWritable k, Iterator<Text> vs, OutputCollector<IntWritable, IntWritable> out, Reporter r)
-				throws IOException {
-			int commentCnt = 0;
-			while(vs.hasNext()) {
-				String v = vs.next().toString();
-				if(v.startsWith("Comment")) {
-					commentCnt++;
-				}
-			}
-			out.collect(k, new IntWritable(commentCnt));
-		}
-		
-		@Override
-		public void configure(final JobConf arg0) { }
-
-		@Override
-		public void close() throws IOException { }
-	}
-	
-	public static class AllCommentCntReducer implements Reducer<IntWritable, Text, IntWritable, IntWritable> {
-		
-		@Override
-		public void reduce(IntWritable k, Iterator<Text> vs, OutputCollector<IntWritable, IntWritable> out, Reporter r)
-				throws IOException {
-			int commentCnt = 0;
-			while(vs.hasNext()) {
-				String v = vs.next().toString();
-				if(v.startsWith("Comment")) {
-					commentCnt++;
-				}
-			}
-			out.collect(new IntWritable(42), new IntWritable(commentCnt));
-		}
-		
-		@Override
-		public void configure(final JobConf arg0) { }
-
-		@Override
-		public void close() throws IOException { }
-	}
-	
-	public static class ConfigurableCntReducer implements Reducer<IntWritable, Text, IntWritable, IntWritable> {
-		private String countPrefix;
-		
-		@Override
-		public void reduce(IntWritable k, Iterator<Text> vs, OutputCollector<IntWritable, IntWritable> out, Reporter r)
-				throws IOException {
-			int commentCnt = 0;
-			while(vs.hasNext()) {
-				String v = vs.next().toString();
-				if(v.startsWith(this.countPrefix)) {
-					commentCnt++;
-				}
-			}
-			out.collect(k, new IntWritable(commentCnt));
-		}
-		
-		@Override
-		public void configure(final JobConf c) { 
-			this.countPrefix = c.get("my.cntPrefix");
-		}
-
-		@Override
-		public void close() throws IOException { }
-	}
-
-	public static class Mapper1 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable, Text>> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Tuple2<IntWritable, Text> map(Tuple2<IntWritable, Text> v)
-		throws Exception {
-			v.f0 = new IntWritable(v.f0.get() / 5);
-			return v;
-		}
-	}
-
-	public static class Mapper2 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable, Text>> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Tuple2<IntWritable, Text> map(Tuple2<IntWritable, Text> v)
-		throws Exception {
-			v.f0 = new IntWritable(v.f0.get() % 5);
-			return v;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java
deleted file mode 100644
index eed6f8f..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.hadoopcompatibility.mapred;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-
-public class HadoopTestData {
-
-	public static DataSet<Tuple2<IntWritable, Text>> getKVPairDataSet(ExecutionEnvironment env) {
-		
-		List<Tuple2<IntWritable, Text>> data = new ArrayList<Tuple2<IntWritable, Text>>();
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(1),new Text("Hi")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(2),new Text("Hello")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(3),new Text("Hello world")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(4),new Text("Hello world, how are you?")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(5),new Text("I am fine.")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(6),new Text("Luke Skywalker")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(7),new Text("Comment#1")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(8),new Text("Comment#2")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(9),new Text("Comment#3")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(10),new Text("Comment#4")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(11),new Text("Comment#5")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(12),new Text("Comment#6")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(13),new Text("Comment#7")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(14),new Text("Comment#8")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(15),new Text("Comment#9")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(16),new Text("Comment#10")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(17),new Text("Comment#11")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(18),new Text("Comment#12")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(19),new Text("Comment#13")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(20),new Text("Comment#14")));
-		data.add(new Tuple2<IntWritable, Text>(new IntWritable(21),new Text("Comment#15")));
-		
-		Collections.shuffle(data);
-		
-		return env.fromCollection(data);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java
deleted file mode 100644
index ce0143a..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.hadoopcompatibility.mapred.example;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopMapFunction;
-import org.apache.flink.api.java.hadoop.mapred.HadoopOutputFormat;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceCombineFunction;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-
-
-
-/**
- * Implements a word count which takes the input file and counts the number of
- * occurrences of each word in the file and writes the result back to disk.
- * 
- * This example shows how to use Hadoop Input Formats, how to convert Hadoop Writables to 
- * common Java types for better usage in a Flink job and how to use Hadoop Output Formats.
- */
-public class HadoopMapredCompatWordCount {
-	
-	public static void main(String[] args) throws Exception {
-		if (args.length < 2) {
-			System.err.println("Usage: WordCount <input path> <result path>");
-			return;
-		}
-		
-		final String inputPath = args[0];
-		final String outputPath = args[1];
-		
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		// Set up the Hadoop Input Format
-		HadoopInputFormat<LongWritable, Text> hadoopInputFormat = new HadoopInputFormat<LongWritable, Text>(new TextInputFormat(), LongWritable.class, Text.class, new JobConf());
-		TextInputFormat.addInputPath(hadoopInputFormat.getJobConf(), new Path(inputPath));
-		
-		// Create a Flink job with it
-		DataSet<Tuple2<LongWritable, Text>> text = env.createInput(hadoopInputFormat);
-		
-		DataSet<Tuple2<Text, LongWritable>> words = 
-				text.flatMap(new HadoopMapFunction<LongWritable, Text, Text, LongWritable>(new Tokenizer()))
-					.groupBy(0).reduceGroup(new HadoopReduceCombineFunction<Text, LongWritable, Text, LongWritable>(new Counter(), new Counter()));
-		
-		// Set up Hadoop Output Format
-		HadoopOutputFormat<Text, LongWritable> hadoopOutputFormat = 
-				new HadoopOutputFormat<Text, LongWritable>(new TextOutputFormat<Text, LongWritable>(), new JobConf());
-		hadoopOutputFormat.getJobConf().set("mapred.textoutputformat.separator", " ");
-		TextOutputFormat.setOutputPath(hadoopOutputFormat.getJobConf(), new Path(outputPath));
-		
-		// Output & Execute
-		words.output(hadoopOutputFormat).setParallelism(1);
-		env.execute("Hadoop Compat WordCount");
-	}
-	
-	
-	public static final class Tokenizer implements Mapper<LongWritable, Text, Text, LongWritable> {
-
-		@Override
-		public void map(LongWritable k, Text v, OutputCollector<Text, LongWritable> out, Reporter rep) 
-				throws IOException {
-			// normalize and split the line
-			String line = v.toString();
-			String[] tokens = line.toLowerCase().split("\\W+");
-			
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new Text(token), new LongWritable(1l));
-				}
-			}
-		}
-		
-		@Override
-		public void configure(JobConf arg0) { }
-		
-		@Override
-		public void close() throws IOException { }
-		
-	}
-	
-	public static final class Counter implements Reducer<Text, LongWritable, Text, LongWritable> {
-
-		@Override
-		public void reduce(Text k, Iterator<LongWritable> vs, OutputCollector<Text, LongWritable> out, Reporter rep)
-				throws IOException {
-			
-			long cnt = 0;
-			while(vs.hasNext()) {
-				cnt += vs.next().get();
-			}
-			out.collect(k, new LongWritable(cnt));
-			
-		}
-		
-		@Override
-		public void configure(JobConf arg0) { }
-		
-		@Override
-		public void close() throws IOException { }
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java
deleted file mode 100644
index 524318c..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.hadoopcompatibility.mapred.wrapper;
-
-import java.util.ArrayList;
-import java.util.NoSuchElementException;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.runtime.WritableSerializer;
-import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator;
-import org.apache.hadoop.io.IntWritable;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class HadoopTupleUnwrappingIteratorTest {
-
-	@Test
-	public void testValueIterator() {
-		
-		HadoopTupleUnwrappingIterator<IntWritable, IntWritable> valIt = 
-				new HadoopTupleUnwrappingIterator<IntWritable, IntWritable>(new WritableSerializer
-						<IntWritable>(IntWritable.class));
-		
-		// many values
-		
-		ArrayList<Tuple2<IntWritable, IntWritable>> tList = new ArrayList<Tuple2<IntWritable, IntWritable>>();
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(1)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(2)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(3)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(4)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(5)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(6)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(7)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(8)));
-		
-		int expectedKey = 1;
-		int[] expectedValues = new int[] {1,2,3,4,5,6,7,8};
-		
-		valIt.set(tList.iterator());
-		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		for(int expectedValue : expectedValues) {
-			Assert.assertTrue(valIt.hasNext());
-			Assert.assertTrue(valIt.hasNext());
-			Assert.assertTrue(valIt.next().get() == expectedValue);
-			Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		}
-		Assert.assertFalse(valIt.hasNext());
-		Assert.assertFalse(valIt.hasNext());
-		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		
-		// one value
-		
-		tList.clear();
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(2),new IntWritable(10)));
-		
-		expectedKey = 2;
-		expectedValues = new int[]{10};
-		
-		valIt.set(tList.iterator());
-		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		for(int expectedValue : expectedValues) {
-			Assert.assertTrue(valIt.hasNext());
-			Assert.assertTrue(valIt.hasNext());
-			Assert.assertTrue(valIt.next().get() == expectedValue);
-			Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		}
-		Assert.assertFalse(valIt.hasNext());
-		Assert.assertFalse(valIt.hasNext());
-		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		
-		// more values
-		
-		tList.clear();
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(3),new IntWritable(10)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(3),new IntWritable(4)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(3),new IntWritable(7)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(3),new IntWritable(9)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(21)));
-		
-		expectedKey = 3;
-		expectedValues = new int[]{10,4,7,9,21};
-		
-		valIt.set(tList.iterator());
-		Assert.assertTrue(valIt.hasNext());
-		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		for(int expectedValue : expectedValues) {
-			Assert.assertTrue(valIt.hasNext());
-			Assert.assertTrue(valIt.hasNext());
-			Assert.assertTrue(valIt.next().get() == expectedValue);
-			Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		}
-		Assert.assertFalse(valIt.hasNext());
-		Assert.assertFalse(valIt.hasNext());
-		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		
-		// no has next calls
-		
-		tList.clear();
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(5)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(8)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(42)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(-1)));
-		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(0)));
-		
-		expectedKey = 4;
-		expectedValues = new int[]{5,8,42,-1,0};
-		
-		valIt.set(tList.iterator());
-		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-		for(int expectedValue : expectedValues) {
-			Assert.assertTrue(valIt.next().get() == expectedValue);
-		}
-		try {
-			valIt.next();
-			Assert.fail();
-		} catch (NoSuchElementException nsee) {
-			// expected
-		}
-		Assert.assertFalse(valIt.hasNext());
-		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
-	}
-	
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java
deleted file mode 100644
index ed83d78..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.hadoopcompatibility.mapreduce.example;
-
-import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat;
-import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat;
-
-/**
- * Implements a word count which takes the input file and counts the number of
- * occurrences of each word in the file and writes the result back to disk.
- * 
- * This example shows how to use Hadoop Input Formats, how to convert Hadoop Writables to 
- * common Java types for better usage in a Flink job and how to use Hadoop Output Formats.
- */
-@SuppressWarnings("serial")
-public class WordCount {
-	
-	public static void main(String[] args) throws Exception {
-		if (args.length < 2) {
-			System.err.println("Usage: WordCount <input path> <result path>");
-			return;
-		}
-		
-		final String inputPath = args[0];
-		final String outputPath = args[1];
-		
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		// Set up the Hadoop Input Format
-		Job job = Job.getInstance();
-		HadoopInputFormat<LongWritable, Text> hadoopInputFormat = new HadoopInputFormat<LongWritable, Text>(new TextInputFormat(), LongWritable.class, Text.class, job);
-		TextInputFormat.addInputPath(job, new Path(inputPath));
-		
-		// Create a Flink job with it
-		DataSet<Tuple2<LongWritable, Text>> text = env.createInput(hadoopInputFormat);
-		
-		// Tokenize the line and convert from Writable "Text" to String for better handling
-		DataSet<Tuple2<String, Integer>> words = text.flatMap(new Tokenizer());
-		
-		// Sum up the words
-		DataSet<Tuple2<String, Integer>> result = words.groupBy(0).aggregate(Aggregations.SUM, 1);
-		
-		// Convert String back to Writable "Text" for use with Hadoop Output Format
-		DataSet<Tuple2<Text, IntWritable>> hadoopResult = result.map(new HadoopDatatypeMapper());
-		
-		// Set up Hadoop Output Format
-		HadoopOutputFormat<Text, IntWritable> hadoopOutputFormat = new HadoopOutputFormat<Text, IntWritable>(new TextOutputFormat<Text, IntWritable>(), job);
-		hadoopOutputFormat.getConfiguration().set("mapreduce.output.textoutputformat.separator", " ");
-		hadoopOutputFormat.getConfiguration().set("mapred.textoutputformat.separator", " "); // set the value for both, since this test
-		TextOutputFormat.setOutputPath(job, new Path(outputPath));
-		
-		// Output & Execute
-		hadoopResult.output(hadoopOutputFormat);
-		env.execute("Word Count");
-	}
-	
-	/**
-	 * Splits a line into words and converts Hadoop Writables into normal Java data types.
-	 */
-	public static final class Tokenizer extends RichFlatMapFunction<Tuple2<LongWritable, Text>, Tuple2<String, Integer>> {
-		
-		@Override
-		public void flatMap(Tuple2<LongWritable, Text> value, Collector<Tuple2<String, Integer>> out) {
-			// normalize and split the line
-			String line = value.f1.toString();
-			String[] tokens = line.toLowerCase().split("\\W+");
-			
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new Tuple2<String, Integer>(token, 1));
-				}
-			}
-		}
-	}
-	
-	/**
-	 * Converts Java data types to Hadoop Writables.
-	 */
-	public static final class HadoopDatatypeMapper extends RichMapFunction<Tuple2<String, Integer>, Tuple2<Text, IntWritable>> {
-		
-		@Override
-		public Tuple2<Text, IntWritable> map(Tuple2<String, Integer> value) throws Exception {
-			return new Tuple2<Text, IntWritable>(new Text(value.f0), new IntWritable(value.f1));
-		}
-		
-	}
-	
-}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/pom.xml b/flink-batch-connectors/flink-hbase/pom.xml
deleted file mode 100644
index 70a5692..0000000
--- a/flink-batch-connectors/flink-hbase/pom.xml
+++ /dev/null
@@ -1,264 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-batch-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-hbase_2.10</artifactId>
-	<name>flink-hbase</name>
-	<packaging>jar</packaging>
-
-	<properties>
-		<hbase.version>1.2.3</hbase.version>
-	</properties>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<version>2.19.1</version>
-				<configuration>
-					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
-					<forkCount>1</forkCount>
-				</configuration>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<!-- Disable inherited shade-flink because of a problem in the shade plugin -->
-						<!-- When enabled you'll run into an infinite loop creating the dependency-reduced-pom.xml -->
-						<!-- Seems similar to https://issues.apache.org/jira/browse/MSHADE-148 -->
-						<id>shade-flink</id>
-						<phase>none</phase>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-
-	<dependencies>
-
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-java</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop2</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-
-			<!--Exclude Guava in order to run the HBaseMiniCluster during testing-->
-			<exclusions>
-				<exclusion>
-					<groupId>com.google.guava</groupId>
-					<artifactId>guava</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<!-- HBase server needed for TableOutputFormat -->
-		<!-- TODO implement bulk output format for HBase -->
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-server</artifactId>
-			<version>${hbase.version}</version>
-			<exclusions>
-				<!-- Remove unneeded dependency, which is conflicting with our jetty-util version. -->
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty-util</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty-sslengine</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-api-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>servlet-api-2.5</artifactId>
-				</exclusion>
-				<!-- The hadoop dependencies are handled through flink-shaded-hadoop -->
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-common</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-auth</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-annotations</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-mapreduce-client-core</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-client</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-hdfs</artifactId>
-				</exclusion>
-				<!-- Bug in hbase annotations, can be removed when fixed. See FLINK-2153. -->
-				<exclusion>
-					<groupId>org.apache.hbase</groupId>
-					<artifactId>hbase-annotations</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<!-- test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-core</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-shaded-include-yarn_2.10</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<!-- Test dependencies are only available for Hadoop-2. -->
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-server</artifactId>
-			<version>${hbase.version}</version>
-			<classifier>tests</classifier>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-minicluster</artifactId>
-			<version>${hadoop.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-hadoop-compat</artifactId>
-			<version>${hbase.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-hdfs</artifactId>
-			<version>${hadoop.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-hadoop2-compat</artifactId>
-			<version>${hbase.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-	</dependencies>
-
-	<profiles>
-		<profile>
-			<id>cdh5.1.3</id>
-			<properties>
-				<hbase.version>0.98.1-cdh5.1.3</hbase.version>
-				<hadoop.version>2.3.0-cdh5.1.3</hadoop.version>
-				<!-- Cloudera use different versions for hadoop core and commons-->
-				<!-- This profile could be removed if Cloudera fix this mismatch! -->
-				<hadoop.core.version>2.3.0-mr1-cdh5.1.3</hadoop.core.version>
-			</properties>
-			<dependencyManagement>
-				<dependencies>
-					<dependency>
-						<groupId>org.apache.hadoop</groupId>
-						<artifactId>hadoop-core</artifactId>
-						<version>${hadoop.core.version}</version>
-					</dependency>
-				</dependencies>
-			</dependencyManagement>
-		</profile>
-
-	</profiles>
-
-</project>


[30/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
new file mode 100644
index 0000000..aa7ea49
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -0,0 +1,2006 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import kafka.consumer.Consumer;
+import kafka.consumer.ConsumerConfig;
+import kafka.consumer.ConsumerIterator;
+import kafka.consumer.KafkaStream;
+import kafka.javaapi.consumer.ConsumerConnector;
+import kafka.message.MessageAndMetadata;
+import kafka.server.KafkaServer;
+import org.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;
+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.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.client.JobCancellationException;
+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.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;
+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.testutils.DataGenerators;
+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;
+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.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema;
+import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
+import org.apache.flink.test.util.SuccessException;
+import org.apache.flink.testutils.junit.RetryOnException;
+import org.apache.flink.testutils.junit.RetryRule;
+import org.apache.flink.util.Collector;
+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 javax.management.MBeanServer;
+import javax.management.ObjectName;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.test.util.TestUtils.tryExecute;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+
+@SuppressWarnings("serial")
+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
+	//
+	//  The tests here are all not activated (by an @Test tag), but need
+	//  to be invoked from the extending classes. That way, the classes can
+	//  select which tests to run.
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Test that ensures the KafkaConsumer is properly failing if the topic doesnt exist
+	 * and a wrong broker was specified
+	 *
+	 * @throws Exception
+	 */
+	public void runFailOnNoBrokerTest() throws Exception {
+		try {
+			Properties properties = new Properties();
+
+			StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			see.getConfig().disableSysoutLogging();
+			see.setRestartStrategy(RestartStrategies.noRestart());
+			see.setParallelism(1);
+
+			// use wrong ports for the consumers
+			properties.setProperty("bootstrap.servers", "localhost:80");
+			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");
+			properties.putAll(secureProps);
+			FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer("doesntexist", new SimpleStringSchema(), properties);
+			DataStream<String> stream = see.addSource(source);
+			stream.print();
+			see.execute("No broker test");
+		} catch(ProgramInvocationException pie) {
+			if(kafkaServer.getVersion().equals("0.9") || kafkaServer.getVersion().equals("0.10")) {
+				assertTrue(pie.getCause() instanceof JobExecutionException);
+
+				JobExecutionException jee = (JobExecutionException) pie.getCause();
+
+				assertTrue(jee.getCause() instanceof TimeoutException);
+
+				TimeoutException te = (TimeoutException) jee.getCause();
+
+				assertEquals("Timeout expired while fetching topic metadata", te.getMessage());
+			} else {
+				assertTrue(pie.getCause() instanceof JobExecutionException);
+
+				JobExecutionException jee = (JobExecutionException) pie.getCause();
+
+				assertTrue(jee.getCause() instanceof RuntimeException);
+
+				RuntimeException re = (RuntimeException) jee.getCause();
+
+				assertTrue(re.getMessage().contains("Unable to retrieve any partitions for the requested topics [doesntexist]"));
+			}
+		}
+	}
+
+	/**
+	 * Ensures that the committed offsets to Kafka are the offsets of "the next record to process"
+	 */
+	public void runCommitOffsetsToKafka() throws Exception {
+		// 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50)
+		final int parallelism = 3;
+		final int recordsInEachPartition = 50;
+
+		final String topicName = writeSequence("testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1);
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.getConfig().disableSysoutLogging();
+		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+		env.setParallelism(parallelism);
+		env.enableCheckpointing(200);
+
+		DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps));
+		stream.addSink(new DiscardingSink<String>());
+
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+		final Thread runner = new Thread("runner") {
+			@Override
+			public void run() {
+				try {
+					env.execute();
+				}
+				catch (Throwable t) {
+					if (!(t.getCause() instanceof JobCancellationException)) {
+						errorRef.set(t);
+					}
+				}
+			}
+		};
+		runner.start();
+
+		final Long l50 = 50L; // the final committed offset in Kafka should be 50
+		final long deadline = 30000 + System.currentTimeMillis();
+
+		KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
+
+		do {
+			Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
+			Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
+			Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
+
+			if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3)) {
+				break;
+			}
+
+			Thread.sleep(100);
+		}
+		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);
+		}
+
+		// final check to see if offsets are correctly in Kafka
+		Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
+		Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
+		Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
+		Assert.assertEquals(Long.valueOf(50L), o1);
+		Assert.assertEquals(Long.valueOf(50L), o2);
+		Assert.assertEquals(Long.valueOf(50L), o3);
+
+		kafkaOffsetHandler.close();
+		deleteTestTopic(topicName);
+	}
+
+	/**
+	 * This test first writes a total of 300 records to a test topic, reads the first 150 so that some offsets are
+	 * committed to Kafka, and then startup the consumer again to read the remaining records starting from the committed offsets.
+	 * The test ensures that whatever offsets were committed to Kafka, the consumer correctly picks them up
+	 * and starts at the correct position.
+	 */
+	public void runStartFromKafkaCommitOffsets() throws Exception {
+		final int parallelism = 3;
+		final int recordsInEachPartition = 300;
+
+		final String topicName = writeSequence("testStartFromKafkaCommitOffsetsTopic", recordsInEachPartition, parallelism, 1);
+
+		KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
+
+		Long o1;
+		Long o2;
+		Long o3;
+		int attempt = 0;
+		// make sure that o1, o2, o3 are not all null before proceeding
+		do {
+			attempt++;
+			LOG.info("Attempt " + attempt + " to read records and commit some offsets to Kafka");
+
+			final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			env.getConfig().disableSysoutLogging();
+			env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+			env.setParallelism(parallelism);
+			env.enableCheckpointing(20); // fast checkpoints to make sure we commit some offsets
+
+			env
+				.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps))
+				.map(new ThrottledMapper<String>(50))
+				.map(new MapFunction<String, Object>() {
+					int count = 0;
+					@Override
+					public Object map(String value) throws Exception {
+						count++;
+						if (count == 150) {
+							throw new SuccessException();
+						}
+						return null;
+					}
+				})
+				.addSink(new DiscardingSink<>());
+
+			tryExecute(env, "Read some records to commit offsets to Kafka");
+
+			o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
+			o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
+			o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
+		} while (o1 == null && o2 == null && o3 == null && attempt < 3);
+
+		if (o1 == null && o2 == null && o3 == null) {
+			throw new RuntimeException("No offsets have been committed after 3 attempts");
+		}
+
+		LOG.info("Got final committed offsets from Kafka o1={}, o2={}, o3={}", o1, o2, o3);
+
+		final StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env2.getConfig().disableSysoutLogging();
+		env2.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+		env2.setParallelism(parallelism);
+
+		// whatever offsets were committed for each partition, the consumer should pick
+		// them up and start from the correct position so that the remaining records are all read
+		HashMap<Integer, Tuple2<Integer, Integer>> partitionsToValuesCountAndStartOffset = new HashMap<>();
+		partitionsToValuesCountAndStartOffset.put(0, new Tuple2<>(
+			(o1 != null) ? (int) (recordsInEachPartition - o1) : recordsInEachPartition,
+			(o1 != null) ? o1.intValue() : 0
+		));
+		partitionsToValuesCountAndStartOffset.put(1, new Tuple2<>(
+			(o2 != null) ? (int) (recordsInEachPartition - o2) : recordsInEachPartition,
+			(o2 != null) ? o2.intValue() : 0
+		));
+		partitionsToValuesCountAndStartOffset.put(2, new Tuple2<>(
+			(o3 != null) ? (int) (recordsInEachPartition - o3) : recordsInEachPartition,
+			(o3 != null) ? o3.intValue() : 0
+		));
+
+		readSequence(env2, standardProps, topicName, partitionsToValuesCountAndStartOffset);
+
+		kafkaOffsetHandler.close();
+		deleteTestTopic(topicName);
+	}
+
+	/**
+	 * This test ensures that when the consumers retrieve some start offset from kafka (earliest, latest), that this offset
+	 * is committed to Kafka, even if some partitions are not read.
+	 *
+	 * Test:
+	 * - Create 3 partitions
+	 * - write 50 messages into each.
+	 * - Start three consumers with auto.offset.reset='latest' and wait until they committed into Kafka.
+	 * - Check if the offsets in Kafka are set to 50 for the three partitions
+	 *
+	 * See FLINK-3440 as well
+	 */
+	public void runAutoOffsetRetrievalAndCommitToKafka() throws Exception {
+		// 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50)
+		final int parallelism = 3;
+		final int recordsInEachPartition = 50;
+
+		final String topicName = writeSequence("testAutoOffsetRetrievalAndCommitToKafkaTopic", recordsInEachPartition, parallelism, 1);
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.getConfig().disableSysoutLogging();
+		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+		env.setParallelism(parallelism);
+		env.enableCheckpointing(200);
+
+		Properties readProps = new Properties();
+		readProps.putAll(standardProps);
+		readProps.setProperty("auto.offset.reset", "latest"); // set to reset to latest, so that partitions are initially not read
+
+		DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), readProps));
+		stream.addSink(new DiscardingSink<String>());
+
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+		final Thread runner = new Thread("runner") {
+			@Override
+			public void run() {
+				try {
+					env.execute();
+				}
+				catch (Throwable t) {
+					if (!(t.getCause() instanceof JobCancellationException)) {
+						errorRef.set(t);
+					}
+				}
+			}
+		};
+		runner.start();
+
+		KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
+
+		final Long l50 = 50L; // the final committed offset in Kafka should be 50
+		final long deadline = 30000 + System.currentTimeMillis();
+		do {
+			Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
+			Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
+			Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
+
+			if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3)) {
+				break;
+			}
+
+			Thread.sleep(100);
+		}
+		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);
+		}
+
+		// final check to see if offsets are correctly in Kafka
+		Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
+		Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
+		Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
+		Assert.assertEquals(Long.valueOf(50L), o1);
+		Assert.assertEquals(Long.valueOf(50L), o2);
+		Assert.assertEquals(Long.valueOf(50L), o3);
+
+		kafkaOffsetHandler.close();
+		deleteTestTopic(topicName);
+	}
+	
+	/**
+	 * Ensure Kafka is working on both producer and consumer side.
+	 * This executes a job that contains two Flink pipelines.
+	 *
+	 * <pre>
+	 * (generator source) --> (kafka sink)-[KAFKA-TOPIC]-(kafka source) --> (validating sink)
+	 * </pre>
+	 * 
+	 * We need to externally retry this test. We cannot let Flink's retry mechanism do it, because the Kafka producer
+	 * does not guarantee exactly-once output. Hence a recovery would introduce duplicates that
+	 * cause the test to fail.
+	 *
+	 * This test also ensures that FLINK-3156 doesn't happen again:
+	 *
+	 * The following situation caused a NPE in the FlinkKafkaConsumer
+	 *
+	 * topic-1 <-- elements are only produced into topic1.
+	 * topic-2
+	 *
+	 * Therefore, this test is consuming as well from an empty topic.
+	 *
+	 */
+	@RetryOnException(times=2, exception=kafka.common.NotLeaderForPartitionException.class)
+	public void runSimpleConcurrentProducerConsumerTopology() throws Exception {
+		final String topic = "concurrentProducerConsumerTopic_" + UUID.randomUUID().toString();
+		final String additionalEmptyTopic = "additionalEmptyTopic_" + UUID.randomUUID().toString();
+
+		final int parallelism = 3;
+		final int elementsPerPartition = 100;
+		final int totalElements = parallelism * elementsPerPartition;
+
+		createTestTopic(topic, parallelism, 2);
+		createTestTopic(additionalEmptyTopic, parallelism, 1); // create an empty topic which will remain empty all the time
+
+		final StreamExecutionEnvironment env =
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(parallelism);
+		env.enableCheckpointing(500);
+		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
+		env.getConfig().disableSysoutLogging();
+
+		TypeInformation<Tuple2<Long, String>> longStringType = TypeInfoParser.parse("Tuple2<Long, String>");
+
+		TypeInformationSerializationSchema<Tuple2<Long, String>> sourceSchema =
+				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
+
+		TypeInformationSerializationSchema<Tuple2<Long, String>> sinkSchema =
+				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
+
+		// ----------- add producer dataflow ----------
+
+		DataStream<Tuple2<Long, String>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Long,String>>() {
+
+			private boolean running = true;
+
+			@Override
+			public void run(SourceContext<Tuple2<Long, String>> ctx) throws InterruptedException {
+				int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition;
+				int limit = cnt + elementsPerPartition;
+
+
+				while (running && cnt < limit) {
+					ctx.collect(new Tuple2<>(1000L + cnt, "kafka-" + cnt));
+					cnt++;
+					// we delay data generation a bit so that we are sure that some checkpoints are
+					// triggered (for FLINK-3156)
+					Thread.sleep(50);
+				}
+			}
+
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		});
+		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
+		producerProperties.setProperty("retries", "3");
+		producerProperties.putAll(secureProps);
+		kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null);
+
+		// ----------- add consumer dataflow ----------
+
+		List<String> topics = new ArrayList<>();
+		topics.add(topic);
+		topics.add(additionalEmptyTopic);
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+		FlinkKafkaConsumerBase<Tuple2<Long, String>> source = kafkaServer.getConsumer(topics, sourceSchema, props);
+
+		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(source).setParallelism(parallelism);
+
+		consuming.addSink(new RichSinkFunction<Tuple2<Long, String>>() {
+
+			private int elCnt = 0;
+			private BitSet validator = new BitSet(totalElements);
+
+			@Override
+			public void invoke(Tuple2<Long, String> value) throws Exception {
+				String[] sp = value.f1.split("-");
+				int v = Integer.parseInt(sp[1]);
+
+				assertEquals(value.f0 - 1000, (long) v);
+
+				assertFalse("Received tuple twice", validator.get(v));
+				validator.set(v);
+				elCnt++;
+
+				if (elCnt == totalElements) {
+					// check if everything in the bitset is set to true
+					int nc;
+					if ((nc = validator.nextClearBit(0)) != totalElements) {
+						fail("The bitset was not set to 1 on all elements. Next clear:"
+								+ nc + " Set: " + validator);
+					}
+					throw new SuccessException();
+				}
+			}
+
+			@Override
+			public void close() throws Exception {
+				super.close();
+			}
+		}).setParallelism(1);
+
+		try {
+			tryExecutePropagateExceptions(env, "runSimpleConcurrentProducerConsumerTopology");
+		}
+		catch (ProgramInvocationException | JobExecutionException e) {
+			// look for NotLeaderForPartitionException
+			Throwable cause = e.getCause();
+
+			// search for nested SuccessExceptions
+			int depth = 0;
+			while (cause != null && depth++ < 20) {
+				if (cause instanceof kafka.common.NotLeaderForPartitionException) {
+					throw (Exception) cause;
+				}
+				cause = cause.getCause();
+			}
+			throw e;
+		}
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests the proper consumption when having a 1:1 correspondence between kafka partitions and
+	 * Flink sources.
+	 */
+	public void runOneToOneExactlyOnceTest() throws Exception {
+
+		final String topic = "oneToOneTopic";
+		final int parallelism = 5;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = parallelism * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
+
+		createTestTopic(topic, parallelism, 1);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				kafkaServer,
+				topic, parallelism, numElementsPerPartition, true);
+
+		// run the topology that fails and recovers
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.enableCheckpointing(500);
+		env.setParallelism(parallelism);
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+		env.getConfig().disableSysoutLogging();
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+
+		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
+
+		env
+				.addSource(kafkaSource)
+				.map(new PartitionValidatingMapper(parallelism, 1))
+				.map(new FailingIdentityMapper<Integer>(failAfterElements))
+				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		FailingIdentityMapper.failedBefore = false;
+		tryExecute(env, "One-to-one exactly once test");
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests the proper consumption when having fewer Flink sources than Kafka partitions, so
+	 * one Flink source will read multiple Kafka partitions.
+	 */
+	public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception {
+		final String topic = "oneToManyTopic";
+		final int numPartitions = 5;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = numPartitions * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
+
+		final int parallelism = 2;
+
+		createTestTopic(topic, numPartitions, 1);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				kafkaServer,
+				topic, numPartitions, numElementsPerPartition, false);
+
+		// run the topology that fails and recovers
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.enableCheckpointing(500);
+		env.setParallelism(parallelism);
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+		env.getConfig().disableSysoutLogging();
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
+
+		env
+				.addSource(kafkaSource)
+				.map(new PartitionValidatingMapper(numPartitions, 3))
+				.map(new FailingIdentityMapper<Integer>(failAfterElements))
+				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		FailingIdentityMapper.failedBefore = false;
+		tryExecute(env, "One-source-multi-partitions exactly once test");
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests the proper consumption when having more Flink sources than Kafka partitions, which means
+	 * that some Flink sources will read no partitions.
+	 */
+	public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception {
+		final String topic = "manyToOneTopic";
+		final int numPartitions = 5;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = numPartitions * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
+
+		final int parallelism = 8;
+
+		createTestTopic(topic, numPartitions, 1);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				kafkaServer,
+				topic, numPartitions, numElementsPerPartition, true);
+
+		// run the topology that fails and recovers
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.enableCheckpointing(500);
+		env.setParallelism(parallelism);
+		// set the number of restarts to one. The failing mapper will fail once, then it's only success exceptions.
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+		env.getConfig().disableSysoutLogging();
+		env.setBufferTimeout(0);
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
+
+		env
+			.addSource(kafkaSource)
+			.map(new PartitionValidatingMapper(numPartitions, 1))
+			.map(new FailingIdentityMapper<Integer>(failAfterElements))
+			.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		FailingIdentityMapper.failedBefore = false;
+		tryExecute(env, "multi-source-one-partitions exactly once test");
+
+
+		deleteTestTopic(topic);
+	}
+	
+	
+	/**
+	 * Tests that the source can be properly canceled when reading full partitions. 
+	 */
+	public void runCancelingOnFullInputTest() throws Exception {
+		final String topic = "cancelingOnFullTopic";
+
+		final int parallelism = 3;
+		createTestTopic(topic, parallelism, 1);
+
+		// launch a producer thread
+		DataGenerators.InfiniteStringsGenerator generator =
+				new DataGenerators.InfiniteStringsGenerator(kafkaServer, topic);
+		generator.start();
+
+		// launch a consumer asynchronously
+
+		final AtomicReference<Throwable> jobError = new AtomicReference<>();
+
+		final Runnable jobRunner = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					env.setParallelism(parallelism);
+					env.enableCheckpointing(100);
+					env.getConfig().disableSysoutLogging();
+
+					Properties props = new Properties();
+					props.putAll(standardProps);
+					props.putAll(secureProps);
+					FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), props);
+
+					env.addSource(source).addSink(new DiscardingSink<String>());
+
+					env.execute("Runner for CancelingOnFullInputTest");
+				}
+				catch (Throwable t) {
+					jobError.set(t);
+				}
+			}
+		};
+
+		Thread runnerThread = new Thread(jobRunner, "program runner thread");
+		runnerThread.start();
+
+		// wait a bit before canceling
+		Thread.sleep(2000);
+
+		Throwable failueCause = jobError.get();
+		if(failueCause != null) {
+			failueCause.printStackTrace();
+			Assert.fail("Test failed prematurely with: " + failueCause.getMessage());
+		}
+
+		// cancel
+		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout), "Runner for CancelingOnFullInputTest");
+
+		// wait for the program to be done and validate that we failed with the right exception
+		runnerThread.join();
+
+		failueCause = jobError.get();
+		assertNotNull("program did not fail properly due to canceling", failueCause);
+		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
+
+		if (generator.isAlive()) {
+			generator.shutdown();
+			generator.join();
+		}
+		else {
+			Throwable t = generator.getError();
+			if (t != null) {
+				t.printStackTrace();
+				fail("Generator failed: " + t.getMessage());
+			} else {
+				fail("Generator failed with no exception");
+			}
+		}
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests that the source can be properly canceled when reading empty partitions. 
+	 */
+	public void runCancelingOnEmptyInputTest() throws Exception {
+		final String topic = "cancelingOnEmptyInputTopic";
+
+		final int parallelism = 3;
+		createTestTopic(topic, parallelism, 1);
+
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final Runnable jobRunner = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					env.setParallelism(parallelism);
+					env.enableCheckpointing(100);
+					env.getConfig().disableSysoutLogging();
+
+					Properties props = new Properties();
+					props.putAll(standardProps);
+					props.putAll(secureProps);
+					FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), props);
+
+					env.addSource(source).addSink(new DiscardingSink<String>());
+
+					env.execute("CancelingOnEmptyInputTest");
+				}
+				catch (Throwable t) {
+					LOG.error("Job Runner failed with exception", t);
+					error.set(t);
+				}
+			}
+		};
+
+		Thread runnerThread = new Thread(jobRunner, "program runner thread");
+		runnerThread.start();
+
+		// wait a bit before canceling
+		Thread.sleep(2000);
+
+		Throwable failueCause = error.get();
+		if (failueCause != null) {
+			failueCause.printStackTrace();
+			Assert.fail("Test failed prematurely with: " + failueCause.getMessage());
+		}
+		// cancel
+		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
+
+		// wait for the program to be done and validate that we failed with the right exception
+		runnerThread.join();
+
+		failueCause = error.get();
+		assertNotNull("program did not fail properly due to canceling", failueCause);
+		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests that the source can be properly canceled when reading full partitions. 
+	 */
+	public void runFailOnDeployTest() throws Exception {
+		final String topic = "failOnDeployTopic";
+
+		createTestTopic(topic, 2, 1);
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(12); // needs to be more that the mini cluster has slots
+		env.getConfig().disableSysoutLogging();
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
+
+		env
+				.addSource(kafkaSource)
+				.addSink(new DiscardingSink<Integer>());
+
+		try {
+			env.execute("test fail on deploy");
+			fail("this test should fail with an exception");
+		}
+		catch (ProgramInvocationException e) {
+
+			// validate that we failed due to a NoResourceAvailableException
+			Throwable cause = e.getCause();
+			int depth = 0;
+			boolean foundResourceException = false;
+
+			while (cause != null && depth++ < 20) {
+				if (cause instanceof NoResourceAvailableException) {
+					foundResourceException = true;
+					break;
+				}
+				cause = cause.getCause();
+			}
+
+			assertTrue("Wrong exception", foundResourceException);
+		}
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Test producing and consuming into multiple topics
+	 * @throws java.lang.Exception
+	 */
+	public void runProduceConsumeMultipleTopics() throws java.lang.Exception {
+		final int NUM_TOPICS = 5;
+		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++) {
+			final String topic = "topic-" + i;
+			topics.add(topic);
+			// create topic
+			createTestTopic(topic, i + 1 /*partitions*/, 1);
+		}
+		// run first job, producing into all topics
+		DataStream<Tuple3<Integer, Integer, String>> stream = env.addSource(new RichParallelSourceFunction<Tuple3<Integer, Integer, String>>() {
+
+			@Override
+			public void run(SourceContext<Tuple3<Integer, Integer, String>> ctx) throws Exception {
+				int partition = getRuntimeContext().getIndexOfThisSubtask();
+
+				for (int topicId = 0; topicId < NUM_TOPICS; topicId++) {
+					for (int i = 0; i < NUM_ELEMENTS; i++) {
+						ctx.collect(new Tuple3<>(partition, i, "topic-" + topicId));
+					}
+				}
+			}
+
+			@Override
+			public void cancel() {
+			}
+		});
+
+		Tuple2WithTopicSchema schema = new Tuple2WithTopicSchema(env.getConfig());
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+		kafkaServer.produceIntoKafka(stream, "dummy", schema, props, null);
+
+		env.execute("Write to topics");
+
+		// run second job consuming from multiple topics
+		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.getConfig().disableSysoutLogging();
+
+		stream = env.addSource(kafkaServer.getConsumer(topics, schema, props));
+
+		stream.flatMap(new FlatMapFunction<Tuple3<Integer, Integer, String>, Integer>() {
+			Map<String, Integer> countPerTopic = new HashMap<>(NUM_TOPICS);
+			@Override
+			public void flatMap(Tuple3<Integer, Integer, String> value, Collector<Integer> out) throws Exception {
+				Integer count = countPerTopic.get(value.f2);
+				if (count == null) {
+					count = 1;
+				} else {
+					count++;
+				}
+				countPerTopic.put(value.f2, count);
+
+				// check map:
+				for (Map.Entry<String, Integer> el: countPerTopic.entrySet()) {
+					if (el.getValue() < NUM_ELEMENTS) {
+						break; // not enough yet
+					}
+					if (el.getValue() > NUM_ELEMENTS) {
+						throw new RuntimeException("There is a failure in the test. I've read " +
+								el.getValue() + " from topic " + el.getKey());
+					}
+				}
+				// we've seen messages from all topics
+				throw new SuccessException();
+			}
+		}).setParallelism(1);
+
+		tryExecute(env, "Count elements from the topics");
+
+
+		// delete all topics again
+		for (int i = 0; i < NUM_TOPICS; i++) {
+			final String topic = "topic-" + i;
+			deleteTestTopic(topic);
+		}
+	}
+
+	/**
+	 * Serialization scheme forwarding byte[] records.
+	 */
+	private static class ByteArraySerializationSchema implements KeyedSerializationSchema<byte[]> {
+
+		@Override
+		public byte[] serializeKey(byte[] element) {
+			return null;
+		}
+
+		@Override
+		public byte[] serializeValue(byte[] element) {
+			return element;
+		}
+
+		@Override
+		public String getTargetTopic(byte[] element) {
+			return null;
+		}
+	}
+
+	private static class Tuple2WithTopicSchema implements KeyedDeserializationSchema<Tuple3<Integer, Integer, String>>,
+			KeyedSerializationSchema<Tuple3<Integer, Integer, String>> {
+
+		private final TypeSerializer<Tuple2<Integer, Integer>> ts;
+		
+		public Tuple2WithTopicSchema(ExecutionConfig ec) {
+			ts = TypeInfoParser.<Tuple2<Integer, Integer>>parse("Tuple2<Integer, Integer>").createSerializer(ec);
+		}
+
+		@Override
+		public Tuple3<Integer, Integer, String> deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
+			DataInputView in = new DataInputViewStreamWrapper(new ByteArrayInputStream(message));
+			Tuple2<Integer, Integer> t2 = ts.deserialize(in);
+			return new Tuple3<>(t2.f0, t2.f1, topic);
+		}
+
+		@Override
+		public boolean isEndOfStream(Tuple3<Integer, Integer, String> nextElement) {
+			return false;
+		}
+
+		@Override
+		public TypeInformation<Tuple3<Integer, Integer, String>> getProducedType() {
+			return TypeInfoParser.parse("Tuple3<Integer, Integer, String>");
+		}
+
+		@Override
+		public byte[] serializeKey(Tuple3<Integer, Integer, String> element) {
+			return null;
+		}
+
+		@Override
+		public byte[] serializeValue(Tuple3<Integer, Integer, String> element) {
+			ByteArrayOutputStream by = new ByteArrayOutputStream();
+			DataOutputView out = new DataOutputViewStreamWrapper(by);
+			try {
+				ts.serialize(new Tuple2<>(element.f0, element.f1), out);
+			} catch (IOException e) {
+				throw new RuntimeException("Error" ,e);
+			}
+			return by.toByteArray();
+		}
+
+		@Override
+		public String getTargetTopic(Tuple3<Integer, Integer, String> element) {
+			return element.f2;
+		}
+	}
+
+	/**
+	 * Test Flink's Kafka integration also with very big records (30MB)
+	 * see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message
+	 *
+	 */
+	public void runBigRecordTestTopology() throws Exception {
+
+		final String topic = "bigRecordTestTopic";
+		final int parallelism = 1; // otherwise, the kafka mini clusters may run out of heap space
+
+		createTestTopic(topic, parallelism, 1);
+
+		final TypeInformation<Tuple2<Long, byte[]>> longBytesInfo = TypeInfoParser.parse("Tuple2<Long, byte[]>");
+
+		final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema =
+				new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setRestartStrategy(RestartStrategies.noRestart());
+		env.getConfig().disableSysoutLogging();
+		env.enableCheckpointing(100);
+		env.setParallelism(parallelism);
+
+		// add consuming topology:
+		Properties consumerProps = new Properties();
+		consumerProps.putAll(standardProps);
+		consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 14));
+		consumerProps.setProperty("max.partition.fetch.bytes", Integer.toString(1024 * 1024 * 14)); // for the new fetcher
+		consumerProps.setProperty("queued.max.message.chunks", "1");
+		consumerProps.putAll(secureProps);
+
+		FlinkKafkaConsumerBase<Tuple2<Long, byte[]>> source = kafkaServer.getConsumer(topic, serSchema, consumerProps);
+		DataStreamSource<Tuple2<Long, byte[]>> consuming = env.addSource(source);
+
+		consuming.addSink(new SinkFunction<Tuple2<Long, byte[]>>() {
+
+			private int elCnt = 0;
+
+			@Override
+			public void invoke(Tuple2<Long, byte[]> value) throws Exception {
+				elCnt++;
+				if (value.f0 == -1) {
+					// we should have seen 11 elements now.
+					if (elCnt == 11) {
+						throw new SuccessException();
+					} else {
+						throw new RuntimeException("There have been "+elCnt+" elements");
+					}
+				}
+				if (elCnt > 10) {
+					throw new RuntimeException("More than 10 elements seen: "+elCnt);
+				}
+			}
+		});
+
+		// add producing topology
+		Properties producerProps = new Properties();
+		producerProps.setProperty("max.request.size", Integer.toString(1024 * 1024 * 15));
+		producerProps.setProperty("retries", "3");
+		producerProps.putAll(secureProps);
+		producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerConnectionStrings);
+
+		DataStream<Tuple2<Long, byte[]>> stream = env.addSource(new RichSourceFunction<Tuple2<Long, byte[]>>() {
+
+			private boolean running;
+
+			@Override
+			public void open(Configuration parameters) throws Exception {
+				super.open(parameters);
+				running = true;
+			}
+
+			@Override
+			public void run(SourceContext<Tuple2<Long, byte[]>> ctx) throws Exception {
+				Random rnd = new Random();
+				long cnt = 0;
+				int sevenMb = 1024 * 1024 * 7;
+
+				while (running) {
+					byte[] wl = new byte[sevenMb + rnd.nextInt(sevenMb)];
+					ctx.collect(new Tuple2<>(cnt++, wl));
+
+					Thread.sleep(100);
+
+					if (cnt == 10) {
+						// signal end
+						ctx.collect(new Tuple2<>(-1L, new byte[]{1}));
+						break;
+					}
+				}
+			}
+
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		});
+
+		kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(serSchema), producerProps, null);
+
+		tryExecute(env, "big topology test");
+		deleteTestTopic(topic);
+	}
+
+	
+	public void runBrokerFailureTest() throws Exception {
+		final String topic = "brokerFailureTestTopic";
+
+		final int parallelism = 2;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = parallelism * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
+
+
+		createTestTopic(topic, parallelism, 2);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				kafkaServer,
+				topic, parallelism, numElementsPerPartition, true);
+
+		// find leader to shut down
+		int leaderId = kafkaServer.getLeaderToShutDown(topic);
+
+		LOG.info("Leader to shutdown {}", leaderId);
+
+
+		// run the topology (the consumers must handle the failures)
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(parallelism);
+		env.enableCheckpointing(500);
+		env.setRestartStrategy(RestartStrategies.noRestart());
+		env.getConfig().disableSysoutLogging();
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, props);
+
+		env
+				.addSource(kafkaSource)
+				.map(new PartitionValidatingMapper(parallelism, 1))
+				.map(new BrokerKillingMapper<Integer>(leaderId, failAfterElements))
+				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		BrokerKillingMapper.killedLeaderBefore = false;
+		tryExecute(env, "Broker failure once test");
+
+		// start a new broker:
+		kafkaServer.restartBroker(leaderId);
+	}
+
+	public void runKeyValueTest() throws Exception {
+		final String topic = "keyvaluetest";
+		createTestTopic(topic, 1, 1);
+		final int ELEMENT_COUNT = 5000;
+
+		// ----------- Write some data into Kafka -------------------
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(1);
+		env.setRestartStrategy(RestartStrategies.noRestart());
+		env.getConfig().disableSysoutLogging();
+
+		DataStream<Tuple2<Long, PojoValue>> kvStream = env.addSource(new SourceFunction<Tuple2<Long, PojoValue>>() {
+			@Override
+			public void run(SourceContext<Tuple2<Long, PojoValue>> ctx) throws Exception {
+				Random rnd = new Random(1337);
+				for (long i = 0; i < ELEMENT_COUNT; i++) {
+					PojoValue pojo = new PojoValue();
+					pojo.when = new Date(rnd.nextLong());
+					pojo.lon = rnd.nextLong();
+					pojo.lat = i;
+					// make every second key null to ensure proper "null" serialization
+					Long key = (i % 2 == 0) ? null : i;
+					ctx.collect(new Tuple2<>(key, pojo));
+				}
+			}
+			@Override
+			public void cancel() {
+			}
+		});
+
+		KeyedSerializationSchema<Tuple2<Long, PojoValue>> schema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig());
+		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
+		producerProperties.setProperty("retries", "3");
+		kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null);
+		env.execute("Write KV to Kafka");
+
+		// ----------- Read the data again -------------------
+
+		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(1);
+		env.setRestartStrategy(RestartStrategies.noRestart());
+		env.getConfig().disableSysoutLogging();
+
+
+		KeyedDeserializationSchema<Tuple2<Long, PojoValue>> readSchema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig());
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+		DataStream<Tuple2<Long, PojoValue>> fromKafka = env.addSource(kafkaServer.getConsumer(topic, readSchema, props));
+		fromKafka.flatMap(new RichFlatMapFunction<Tuple2<Long,PojoValue>, Object>() {
+			long counter = 0;
+			@Override
+			public void flatMap(Tuple2<Long, PojoValue> value, Collector<Object> out) throws Exception {
+				// the elements should be in order.
+				Assert.assertTrue("Wrong value " + value.f1.lat, value.f1.lat == counter );
+				if (value.f1.lat % 2 == 0) {
+					assertNull("key was not null", value.f0);
+				} else {
+					Assert.assertTrue("Wrong value " + value.f0, value.f0 == counter);
+				}
+				counter++;
+				if (counter == ELEMENT_COUNT) {
+					// we got the right number of elements
+					throw new SuccessException();
+				}
+			}
+		});
+
+		tryExecute(env, "Read KV from Kafka");
+
+		deleteTestTopic(topic);
+	}
+
+	public static class PojoValue {
+		public Date when;
+		public long lon;
+		public long lat;
+		public PojoValue() {}
+	}
+
+
+	/**
+	 * Test delete behavior and metrics for producer
+	 * @throws Exception
+	 */
+	public void runAllDeletesTest() throws Exception {
+		final String topic = "alldeletestest";
+		createTestTopic(topic, 1, 1);
+		final int ELEMENT_COUNT = 300;
+
+		// ----------- Write some data into Kafka -------------------
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(1);
+		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+		env.getConfig().disableSysoutLogging();
+
+		DataStream<Tuple2<byte[], PojoValue>> kvStream = env.addSource(new SourceFunction<Tuple2<byte[], PojoValue>>() {
+			@Override
+			public void run(SourceContext<Tuple2<byte[], PojoValue>> ctx) throws Exception {
+				Random rnd = new Random(1337);
+				for (long i = 0; i < ELEMENT_COUNT; i++) {
+					final byte[] key = new byte[200];
+					rnd.nextBytes(key);
+					ctx.collect(new Tuple2<>(key, (PojoValue) null));
+				}
+			}
+			@Override
+			public void cancel() {
+			}
+		});
+
+		TypeInformationKeyValueSerializationSchema<byte[], PojoValue> schema = new TypeInformationKeyValueSerializationSchema<>(byte[].class, PojoValue.class, env.getConfig());
+
+		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
+		producerProperties.setProperty("retries", "3");
+		producerProperties.putAll(secureProps);
+		kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null);
+
+		env.execute("Write deletes to Kafka");
+
+		// ----------- Read the data again -------------------
+
+		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(1);
+		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+		env.getConfig().disableSysoutLogging();
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+		DataStream<Tuple2<byte[], PojoValue>> fromKafka = env.addSource(kafkaServer.getConsumer(topic, schema, props));
+
+		fromKafka.flatMap(new RichFlatMapFunction<Tuple2<byte[], PojoValue>, Object>() {
+			long counter = 0;
+			@Override
+			public void flatMap(Tuple2<byte[], PojoValue> value, Collector<Object> out) throws Exception {
+				// ensure that deleted messages are passed as nulls
+				assertNull(value.f1);
+				counter++;
+				if (counter == ELEMENT_COUNT) {
+					// we got the right number of elements
+					throw new SuccessException();
+				}
+			}
+		});
+
+		tryExecute(env, "Read deletes from Kafka");
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Test that ensures that DeserializationSchema.isEndOfStream() is properly evaluated.
+	 *
+	 * @throws Exception
+	 */
+	public void runEndOfStreamTest() throws Exception {
+
+		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);
+		env1.setParallelism(1);
+		env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+		env1.getConfig().disableSysoutLogging();
+
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+
+		DataStream<Tuple2<Integer, Integer>> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(ELEMENT_COUNT), props));
+		fromKafka.flatMap(new FlatMapFunction<Tuple2<Integer,Integer>, Void>() {
+			@Override
+			public void flatMap(Tuple2<Integer, Integer> value, Collector<Void> out) throws Exception {
+				// noop ;)
+			}
+		});
+
+		JobExecutionResult result = tryExecute(env1, "Consume " + ELEMENT_COUNT + " elements from Kafka");
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Test metrics reporting for consumer
+	 *
+	 * @throws Exception
+	 */
+	public void runMetricsTest() throws Throwable {
+
+		// create a stream with 5 topics
+		final String topic = "metricsStream";
+		createTestTopic(topic, 5, 1);
+
+		final Tuple1<Throwable> error = new Tuple1<>(null);
+		Runnable job = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					// start job writing & reading data.
+					final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					env1.setParallelism(1);
+					env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+					env1.getConfig().disableSysoutLogging();
+					env1.disableOperatorChaining(); // let the source read everything into the network buffers
+
+					Properties props = new Properties();
+					props.putAll(standardProps);
+					props.putAll(secureProps);
+
+					TypeInformationSerializationSchema<Tuple2<Integer, Integer>> schema = new TypeInformationSerializationSchema<>(TypeInfoParser.<Tuple2<Integer, Integer>>parse("Tuple2<Integer, Integer>"), env1.getConfig());
+					DataStream<Tuple2<Integer, Integer>> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, schema, standardProps));
+					fromKafka.flatMap(new FlatMapFunction<Tuple2<Integer, Integer>, Void>() {
+						@Override
+						public void flatMap(Tuple2<Integer, Integer> value, Collector<Void> out) throws Exception {// no op
+						}
+					});
+
+					DataStream<Tuple2<Integer, Integer>> fromGen = env1.addSource(new RichSourceFunction<Tuple2<Integer, Integer>>() {
+						boolean running = true;
+
+						@Override
+						public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
+							int i = 0;
+							while (running) {
+								ctx.collect(Tuple2.of(i++, getRuntimeContext().getIndexOfThisSubtask()));
+								Thread.sleep(1);
+							}
+						}
+
+						@Override
+						public void cancel() {
+							running = false;
+						}
+					});
+
+					kafkaServer.produceIntoKafka(fromGen, topic, new KeyedSerializationSchemaWrapper<>(schema), standardProps, null);
+
+					env1.execute("Metrics test job");
+				} catch(Throwable t) {
+					LOG.warn("Got exception during execution", t);
+					if(!(t.getCause() instanceof JobCancellationException)) { // we'll cancel the job
+						error.f0 = t;
+					}
+				}
+			}
+		};
+		Thread jobThread = new Thread(job);
+		jobThread.start();
+
+		try {
+			// connect to JMX
+			MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer();
+			// wait until we've found all 5 offset metrics
+			Set<ObjectName> offsetMetrics = mBeanServer.queryNames(new ObjectName("*current-offsets*:*"), null);
+			while (offsetMetrics.size() < 5) { // test will time out if metrics are not properly working
+				if (error.f0 != null) {
+					// fail test early
+					throw error.f0;
+				}
+				offsetMetrics = mBeanServer.queryNames(new ObjectName("*current-offsets*:*"), null);
+				Thread.sleep(50);
+			}
+			Assert.assertEquals(5, offsetMetrics.size());
+			// we can't rely on the consumer to have touched all the partitions already
+			// that's why we'll wait until all five partitions have a positive offset.
+			// The test will fail if we never meet the condition
+			while (true) {
+				int numPosOffsets = 0;
+				// check that offsets are correctly reported
+				for (ObjectName object : offsetMetrics) {
+					Object offset = mBeanServer.getAttribute(object, "Value");
+					if((long) offset >= 0) {
+						numPosOffsets++;
+					}
+				}
+				if (numPosOffsets == 5) {
+					break;
+				}
+				// wait for the consumer to consume on all partitions
+				Thread.sleep(50);
+			}
+
+			// check if producer metrics are also available.
+			Set<ObjectName> producerMetrics = mBeanServer.queryNames(new ObjectName("*KafkaProducer*:*"), null);
+			Assert.assertTrue("No producer metrics found", producerMetrics.size() > 30);
+
+
+			LOG.info("Found all JMX metrics. Cancelling job.");
+		} finally {
+			// cancel
+			JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
+		}
+
+		while (jobThread.isAlive()) {
+			Thread.sleep(50);
+		}
+		if (error.f0 != null) {
+			throw error.f0;
+		}
+
+		deleteTestTopic(topic);
+	}
+
+
+	public static class FixedNumberDeserializationSchema implements DeserializationSchema<Tuple2<Integer, Integer>> {
+		
+		final int finalCount;
+		int count = 0;
+		
+		TypeInformation<Tuple2<Integer, Integer>> ti = TypeInfoParser.parse("Tuple2<Integer, Integer>");
+		TypeSerializer<Tuple2<Integer, Integer>> ser = ti.createSerializer(new ExecutionConfig());
+
+		public FixedNumberDeserializationSchema(int finalCount) {
+			this.finalCount = finalCount;
+		}
+
+		@Override
+		public Tuple2<Integer, Integer> deserialize(byte[] message) throws IOException {
+			DataInputView in = new DataInputViewStreamWrapper(new ByteArrayInputStream(message));
+			return ser.deserialize(in);
+		}
+
+		@Override
+		public boolean isEndOfStream(Tuple2<Integer, Integer> nextElement) {
+			return ++count >= finalCount;
+		}
+
+		@Override
+		public TypeInformation<Tuple2<Integer, Integer>> getProducedType() {
+			return ti;
+		}
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  Reading writing test data sets
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Runs a job using the provided environment to read a sequence of records from a single Kafka topic.
+	 * The method allows to individually specify the expected starting offset and total read value count of each partition.
+	 * The job will be considered successful only if all partition read results match the start offset and value count criteria.
+	 */
+	protected void readSequence(StreamExecutionEnvironment env, Properties cc,
+								final String topicName,
+								final Map<Integer, Tuple2<Integer, Integer>> partitionsToValuesCountAndStartOffset) throws Exception {
+		final int sourceParallelism = partitionsToValuesCountAndStartOffset.keySet().size();
+
+		int finalCountTmp = 0;
+		for (Map.Entry<Integer, Tuple2<Integer, Integer>> valuesCountAndStartOffset : partitionsToValuesCountAndStartOffset.entrySet()) {
+			finalCountTmp += valuesCountAndStartOffset.getValue().f0;
+		}
+		final int finalCount = finalCountTmp;
+
+		final TypeInformation<Tuple2<Integer, Integer>> intIntTupleType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
+
+		final TypeInformationSerializationSchema<Tuple2<Integer, Integer>> deser =
+			new TypeInformationSerializationSchema<>(intIntTupleType, env.getConfig());
+
+		// create the consumer
+		cc.putAll(secureProps);
+		FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> consumer = kafkaServer.getConsumer(topicName, deser, cc);
+
+		DataStream<Tuple2<Integer, Integer>> source = env
+			.addSource(consumer).setParallelism(sourceParallelism)
+			.map(new ThrottledMapper<Tuple2<Integer, Integer>>(20)).setParallelism(sourceParallelism);
+
+		// verify data
+		source.flatMap(new RichFlatMapFunction<Tuple2<Integer, Integer>, Integer>() {
+
+			private HashMap<Integer, BitSet> partitionsToValueCheck;
+			private int count = 0;
+
+			@Override
+			public void open(Configuration parameters) throws Exception {
+				partitionsToValueCheck = new HashMap<>();
+				for (Integer partition : partitionsToValuesCountAndStartOffset.keySet()) {
+					partitionsToValueCheck.put(partition, new BitSet());
+				}
+			}
+
+			@Override
+			public void flatMap(Tuple2<Integer, Integer> value, Collector<Integer> out) throws Exception {
+				int partition = value.f0;
+				int val = value.f1;
+
+				BitSet bitSet = partitionsToValueCheck.get(partition);
+				if (bitSet == null) {
+					throw new RuntimeException("Got a record from an unknown partition");
+				} else {
+					bitSet.set(val - partitionsToValuesCountAndStartOffset.get(partition).f1);
+				}
+
+				count++;
+
+				LOG.info("Received message {}, total {} messages", value, count);
+
+				// verify if we've seen everything
+				if (count == finalCount) {
+					for (Map.Entry<Integer, BitSet> partitionsToValueCheck : this.partitionsToValueCheck.entrySet()) {
+						BitSet check = partitionsToValueCheck.getValue();
+						int expectedValueCount = partitionsToValuesCountAndStartOffset.get(partitionsToValueCheck.getKey()).f0;
+
+						if (check.cardinality() != expectedValueCount) {
+							throw new RuntimeException("Expected cardinality to be " + expectedValueCount +
+								", but was " + check.cardinality());
+						} else if (check.nextClearBit(0) != expectedValueCount) {
+							throw new RuntimeException("Expected next clear bit to be " + expectedValueCount +
+								", but was " + check.cardinality());
+						}
+					}
+
+					// test has passed
+					throw new SuccessException();
+				}
+			}
+
+		}).setParallelism(1);
+
+		tryExecute(env, "Read data from Kafka");
+
+		LOG.info("Successfully read sequence for verification");
+	}
+
+	/**
+	 * Variant of {@link KafkaConsumerTestBase#readSequence(StreamExecutionEnvironment, Properties, String, Map)} to
+	 * expect reading from the same start offset and the same value count for all partitions of a single Kafka topic.
+	 */
+	protected void readSequence(StreamExecutionEnvironment env, Properties cc,
+								final int sourceParallelism,
+								final String topicName,
+								final int valuesCount, final int startFrom) throws Exception {
+		HashMap<Integer, Tuple2<Integer, Integer>> partitionsToValuesCountAndStartOffset = new HashMap<>();
+		for (int i = 0; i < sourceParallelism; i++) {
+			partitionsToValuesCountAndStartOffset.put(i, new Tuple2<>(valuesCount, startFrom));
+		}
+		readSequence(env, cc, topicName, partitionsToValuesCountAndStartOffset);
+	}
+
+	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" +
+				"===================================");
+
+		final TypeInformation<Tuple2<Integer, Integer>> resultType = 
+				TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {});
+
+		final KeyedSerializationSchema<Tuple2<Integer, Integer>> serSchema =
+				new KeyedSerializationSchemaWrapper<>(
+						new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig()));
+
+		final KeyedDeserializationSchema<Tuple2<Integer, Integer>> deserSchema =
+				new KeyedDeserializationSchemaWrapper<>(
+						new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig()));
+		
+		final int maxNumAttempts = 10;
+
+		for (int attempt = 1; attempt <= maxNumAttempts; attempt++) {
+			
+			final String topicName = baseTopicName + '-' + attempt;
+			
+			LOG.info("Writing attempt #1");
+			
+			// -------- Write the Sequence --------
+
+			createTestTopic(topicName, parallelism, replicationFactor);
+
+			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");
+			producerProperties.putAll(secureProps);
+			
+			kafkaServer.produceIntoKafka(stream, topicName, serSchema, producerProperties, new Tuple2Partitioner(parallelism))
+					.setParallelism(parallelism);
+
+			try {
+				writeEnv.execute("Write sequence");
+			}
+			catch (Exception e) {
+				LOG.error("Write attempt failed, trying again", e);
+				deleteTestTopic(topicName);
+				JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout));
+				continue;
+			}
+			
+			LOG.info("Finished writing sequence");
+
+			// -------- Validate the Sequence --------
+			
+			// we need to validate the sequence, because kafka's producers are not exactly once
+			LOG.info("Validating sequence");
+
+			JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout));
+			
+			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");
+			readProps.putAll(secureProps);
+			FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> consumer = kafkaServer.getConsumer(topicName, deserSchema, readProps);
+
+			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;
+				}
+			}
+
+			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");
+	}
+
+	// ------------------------------------------------------------------------
+	//  Debugging utilities
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Read topic to list, only using Kafka code.
+	 */
+	private static List<MessageAndMetadata<byte[], byte[]>> readTopicToList(String topicName, ConsumerConfig config, final int stopAfter) {
+		ConsumerConnector consumerConnector = Consumer.createJavaConsumerConnector(config);
+		// we request only one stream per consumer instance. Kafka will make sure that each consumer group
+		// will see each message only once.
+		Map<String,Integer> topicCountMap = Collections.singletonMap(topicName, 1);
+		Map<String, List<KafkaStream<byte[], byte[]>>> streams = consumerConnector.createMessageStreams(topicCountMap);
+		if (streams.size() != 1) {
+			throw new RuntimeException("Expected only one message stream but got "+streams.size());
+		}
+		List<KafkaStream<byte[], byte[]>> kafkaStreams = streams.get(topicName);
+		if (kafkaStreams == null) {
+			throw new RuntimeException("Requested stream not available. Available streams: "+streams.toString());
+		}
+		if (kafkaStreams.size() != 1) {
+			throw new RuntimeException("Requested 1 stream from Kafka, bot got "+kafkaStreams.size()+" streams");
+		}
+		LOG.info("Opening Consumer instance for topic '{}' on group '{}'", topicName, config.groupId());
+		ConsumerIterator<byte[], byte[]> iteratorToRead = kafkaStreams.get(0).iterator();
+
+		List<MessageAndMetadata<byte[], byte[]>> result = new ArrayList<>();
+		int read = 0;
+		while(iteratorToRead.hasNext()) {
+			read++;
+			result.add(iteratorToRead.next());
+			if (read == stopAfter) {
+				LOG.info("Read "+read+" elements");
+				return result;
+			}
+		}
+		return result;
+	}
+
+	private static void printTopic(String topicName, ConsumerConfig config,
+								DeserializationSchema<?> deserializationSchema,
+								int stopAfter) throws IOException {
+
+		List<MessageAndMetadata<byte[], byte[]>> contents = readTopicToList(topicName, config, stopAfter);
+		LOG.info("Printing contents of topic {} in consumer grouo {}", topicName, config.groupId());
+
+		for (MessageAndMetadata<byte[], byte[]> message: contents) {
+			Object out = deserializationSchema.deserialize(message.message());
+			LOG.info("Message: partition: {} offset: {} msg: {}", message.partition(), message.offset(), out.toString());
+		}
+	}
+
+	private static void printTopic(String topicName, int elements,DeserializationSchema<?> deserializer) 
+			throws IOException
+	{
+		// write the sequence to log for debugging purposes
+		Properties newProps = new Properties(standardProps);
+		newProps.setProperty("group.id", "topic-printer"+ UUID.randomUUID().toString());
+		newProps.setProperty("auto.offset.reset", "smallest");
+		newProps.setProperty("zookeeper.connect", standardProps.getProperty("zookeeper.connect"));
+		newProps.putAll(secureProps);
+
+		ConsumerConfig printerConfig = new ConsumerConfig(newProps);
+		printTopic(topicName, printerConfig, deserializer, elements);
+	}
+
+
+	public static class BrokerKillingMapper<T> extends RichMapFunction<T,T>
+			implements Checkpointed<Integer>, CheckpointListener {
+
+		private static final long serialVersionUID = 6334389850158707313L;
+
+		public static volatile boolean killedLeaderBefore;
+		public static volatile boolean hasBeenCheckpointedBeforeFailure;
+		
+		private final int shutdownBrokerId;
+		private final int failCount;
+		private int numElementsTotal;
+
+		private boolean failer;
+		private boolean hasBeenCheckpointed;
+
+
+		public BrokerKillingMapper(int shutdownBrokerId, int failCount) {
+			this.shutdownBrokerId = shutdownBrokerId;
+			this.failCount = failCount;
+		}
+
+		@Override
+		public void open(Configuration parameters) {
+			failer = getRuntimeContext().getIndexOfThisSubtask() == 0;
+		}
+
+		@Override
+		public T map(T value) throws Exception {
+			numElementsTotal++;
+			
+			if (!killedLeaderBefore) {
+				Thread.sleep(10);
+				
+				if (failer && numElementsTotal >= failCount) {
+					// shut down a Kafka broker
+					KafkaServer toShutDown = null;
+					for (KafkaServer server : kafkaServer.getBrokers()) {
+
+						if (kafkaServer.getBrokerId(server) == shutdownBrokerId) {
+							toShutDown = server;
+							break;
+						}
+					}
+	
+					if (toShutDown == null) {
+						StringBuilder listOfBrokers = new StringBuilder();
+						for (KafkaServer server : kafkaServer.getBrokers()) {
+							listOfBrokers.append(kafkaServer.getBrokerId(server));
+							listOfBrokers.append(" ; ");
+						}
+						
+						throw new Exception("Cannot find broker to shut down: " + shutdownBrokerId
+								+ " ; available brokers: " + listOfBrokers.toString());
+					}
+					else {
+						hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed;
+						killedLeaderBefore = true;
+						toShutDown.shutdown();
+					}
+				}
+			}
+			return value;
+		}
+
+		@Override
+		public void notifyCheckpointComplete(long checkpointId) {
+			hasBeenCheckpointed = true;
+		}
+
+		@Override
+		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+			return numElementsTotal;
+		}
+
+		@Override
+		public void restoreState(Integer state) {
+			this.numElementsTotal = state;
+		}
+	}
+}

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


[23/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java
new file mode 100644
index 0000000..dc5396a
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis.common.container;
+
+import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisClusterConfig;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisConfigBase;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisPoolConfig;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisSentinelConfig;
+import org.apache.flink.util.Preconditions;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.JedisPool;
+import redis.clients.jedis.JedisSentinelPool;
+
+/**
+ * The builder for {@link RedisCommandsContainer}.
+ */
+public class RedisCommandsContainerBuilder {
+
+	/**
+	 * Initialize the {@link RedisCommandsContainer} based on the instance type.
+	 * @param flinkJedisConfigBase configuration base
+	 * @return @throws IllegalArgumentException if jedisPoolConfig, jedisClusterConfig and jedisSentinelConfig are all null
+     */
+	public static RedisCommandsContainer build(FlinkJedisConfigBase flinkJedisConfigBase){
+		if(flinkJedisConfigBase instanceof FlinkJedisPoolConfig){
+			FlinkJedisPoolConfig flinkJedisPoolConfig = (FlinkJedisPoolConfig) flinkJedisConfigBase;
+			return RedisCommandsContainerBuilder.build(flinkJedisPoolConfig);
+		} else if (flinkJedisConfigBase instanceof FlinkJedisClusterConfig) {
+			FlinkJedisClusterConfig flinkJedisClusterConfig = (FlinkJedisClusterConfig) flinkJedisConfigBase;
+			return RedisCommandsContainerBuilder.build(flinkJedisClusterConfig);
+		} else if (flinkJedisConfigBase instanceof FlinkJedisSentinelConfig) {
+			FlinkJedisSentinelConfig flinkJedisSentinelConfig = (FlinkJedisSentinelConfig) flinkJedisConfigBase;
+			return RedisCommandsContainerBuilder.build(flinkJedisSentinelConfig);
+		} else {
+			throw new IllegalArgumentException("Jedis configuration not found");
+		}
+	}
+
+	/**
+	 * Builds container for single Redis environment.
+	 *
+	 * @param jedisPoolConfig configuration for JedisPool
+	 * @return container for single Redis environment
+	 * @throws NullPointerException if jedisPoolConfig is null
+	 */
+	public static RedisCommandsContainer build(FlinkJedisPoolConfig jedisPoolConfig) {
+		Preconditions.checkNotNull(jedisPoolConfig, "Redis pool config should not be Null");
+
+		GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig();
+		genericObjectPoolConfig.setMaxIdle(jedisPoolConfig.getMaxIdle());
+		genericObjectPoolConfig.setMaxTotal(jedisPoolConfig.getMaxTotal());
+		genericObjectPoolConfig.setMinIdle(jedisPoolConfig.getMinIdle());
+
+		JedisPool jedisPool = new JedisPool(genericObjectPoolConfig, jedisPoolConfig.getHost(),
+			jedisPoolConfig.getPort(), jedisPoolConfig.getConnectionTimeout(), jedisPoolConfig.getPassword(),
+			jedisPoolConfig.getDatabase());
+		return new RedisContainer(jedisPool);
+	}
+
+	/**
+	 * Builds container for Redis Cluster environment.
+	 *
+	 * @param jedisClusterConfig configuration for JedisCluster
+	 * @return container for Redis Cluster environment
+	 * @throws NullPointerException if jedisClusterConfig is null
+	 */
+	public static RedisCommandsContainer build(FlinkJedisClusterConfig jedisClusterConfig) {
+		Preconditions.checkNotNull(jedisClusterConfig, "Redis cluster config should not be Null");
+
+		GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig();
+		genericObjectPoolConfig.setMaxIdle(jedisClusterConfig.getMaxIdle());
+		genericObjectPoolConfig.setMaxTotal(jedisClusterConfig.getMaxTotal());
+		genericObjectPoolConfig.setMinIdle(jedisClusterConfig.getMinIdle());
+
+		JedisCluster jedisCluster = new JedisCluster(jedisClusterConfig.getNodes(), jedisClusterConfig.getConnectionTimeout(),
+			jedisClusterConfig.getMaxRedirections(), genericObjectPoolConfig);
+		return new RedisClusterContainer(jedisCluster);
+	}
+
+	/**
+	 * Builds container for Redis Sentinel environment.
+	 *
+	 * @param jedisSentinelConfig configuration for JedisSentinel
+	 * @return container for Redis sentinel environment
+	 * @throws NullPointerException if jedisSentinelConfig is null
+	 */
+	public static RedisCommandsContainer build(FlinkJedisSentinelConfig jedisSentinelConfig) {
+		Preconditions.checkNotNull(jedisSentinelConfig, "Redis sentinel config should not be Null");
+
+		GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig();
+		genericObjectPoolConfig.setMaxIdle(jedisSentinelConfig.getMaxIdle());
+		genericObjectPoolConfig.setMaxTotal(jedisSentinelConfig.getMaxTotal());
+		genericObjectPoolConfig.setMinIdle(jedisSentinelConfig.getMinIdle());
+
+		JedisSentinelPool jedisSentinelPool = new JedisSentinelPool(jedisSentinelConfig.getMasterName(),
+			jedisSentinelConfig.getSentinels(), genericObjectPoolConfig,
+			jedisSentinelConfig.getConnectionTimeout(), jedisSentinelConfig.getSoTimeout(),
+			jedisSentinelConfig.getPassword(), jedisSentinelConfig.getDatabase());
+		return new RedisContainer(jedisSentinelPool);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisContainer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisContainer.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisContainer.java
new file mode 100644
index 0000000..ba4bbda
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisContainer.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis.common.container;
+
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisPool;
+import redis.clients.jedis.JedisSentinelPool;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Redis command container if we want to connect to a single Redis server or to Redis sentinels
+ * If want to connect to a single Redis server, please use the first constructor {@link #RedisContainer(JedisPool)}.
+ * If want to connect to a Redis sentinels, please use the second constructor {@link #RedisContainer(JedisSentinelPool)}
+ */
+public class RedisContainer implements RedisCommandsContainer, Closeable {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(RedisContainer.class);
+
+	private transient JedisPool jedisPool;
+	private transient JedisSentinelPool jedisSentinelPool;
+
+	/**
+	 * Use this constructor if to connect with single Redis server.
+	 *
+	 * @param jedisPool JedisPool which actually manages Jedis instances
+	 */
+	public RedisContainer(JedisPool jedisPool) {
+		Preconditions.checkNotNull(jedisPool, "Jedis Pool can not be null");
+		this.jedisPool = jedisPool;
+		this.jedisSentinelPool = null;
+	}
+
+	/**
+	 * Use this constructor if Redis environment is clustered with sentinels.
+	 *
+	 * @param sentinelPool SentinelPool which actually manages Jedis instances
+	 */
+	public RedisContainer(final JedisSentinelPool sentinelPool) {
+		Preconditions.checkNotNull(sentinelPool, "Jedis Sentinel Pool can not be null");
+		this.jedisPool = null;
+		this.jedisSentinelPool = sentinelPool;
+	}
+
+	/**
+	 * Closes the Jedis instances.
+	 */
+	@Override
+	public void close() throws IOException {
+		if (this.jedisPool != null) {
+			this.jedisPool.close();
+		}
+		if (this.jedisSentinelPool != null) {
+			this.jedisSentinelPool.close();
+		}
+	}
+
+	@Override
+	public void open() throws Exception {
+
+		// echo() tries to open a connection and echos back the
+		// message passed as argument. Here we use it to monitor
+		// if we can communicate with the cluster.
+
+		getInstance().echo("Test");
+	}
+
+	@Override
+	public void hset(final String key, final String hashField, final String value) {
+		Jedis jedis = null;
+		try {
+			jedis = getInstance();
+			jedis.hset(key, hashField, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command HSET to key {} and hashField {} error message {}",
+					key, hashField, e.getMessage());
+			}
+			throw e;
+		} finally {
+			releaseInstance(jedis);
+		}
+	}
+
+	@Override
+	public void rpush(final String listName, final String value) {
+		Jedis jedis = null;
+		try {
+			jedis = getInstance();
+			jedis.rpush(listName, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command RPUSH to list {} error message {}",
+					listName, e.getMessage());
+			}
+			throw e;
+		} finally {
+			releaseInstance(jedis);
+		}
+	}
+
+	@Override
+	public void lpush(String listName, String value) {
+		Jedis jedis = null;
+		try {
+			jedis = getInstance();
+			jedis.lpush(listName, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command LUSH to list {} error message {}",
+					listName, e.getMessage());
+			}
+			throw e;
+		} finally {
+			releaseInstance(jedis);
+		}
+	}
+
+	@Override
+	public void sadd(final String setName, final String value) {
+		Jedis jedis = null;
+		try {
+			jedis = getInstance();
+			jedis.sadd(setName, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command RPUSH to set {} error message {}",
+					setName, e.getMessage());
+			}
+			throw e;
+		} finally {
+			releaseInstance(jedis);
+		}
+	}
+
+	@Override
+	public void publish(final String channelName, final String message) {
+		Jedis jedis = null;
+		try {
+			jedis = getInstance();
+			jedis.publish(channelName, message);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command PUBLISH to channel {} error message {}",
+					channelName, e.getMessage());
+			}
+			throw e;
+		} finally {
+			releaseInstance(jedis);
+		}
+	}
+
+	@Override
+	public void set(final String key, final String value) {
+		Jedis jedis = null;
+		try {
+			jedis = getInstance();
+			jedis.set(key, value);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command SET to key {} error message {}",
+					key, e.getMessage());
+			}
+			throw e;
+		} finally {
+			releaseInstance(jedis);
+		}
+	}
+
+	@Override
+	public void pfadd(final String key, final String element) {
+		Jedis jedis = null;
+		try {
+			jedis = getInstance();
+			jedis.pfadd(key, element);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command PFADD to key {} error message {}",
+					key, e.getMessage());
+			}
+			throw e;
+		} finally {
+			releaseInstance(jedis);
+		}
+	}
+
+	@Override
+	public void zadd(final String key, final String score, final String element) {
+		Jedis jedis = null;
+		try {
+			jedis = getInstance();
+			jedis.zadd(key, Double.valueOf(score), element);
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send Redis message with command ZADD to set {} error message {}",
+					key, e.getMessage());
+			}
+			throw e;
+		} finally {
+			releaseInstance(jedis);
+		}
+	}
+
+	/**
+	 * Returns Jedis instance from the pool.
+	 *
+	 * @return the Jedis instance
+     */
+	private Jedis getInstance() {
+		if (jedisSentinelPool != null) {
+			return jedisSentinelPool.getResource();
+		} else {
+			return jedisPool.getResource();
+		}
+	}
+
+	/**
+	 * Closes the jedis instance after finishing the command.
+	 *
+	 * @param jedis The jedis instance
+     */
+	private void releaseInstance(final Jedis jedis) {
+		if (jedis == null) {
+			return;
+		}
+		try {
+			jedis.close();
+		} catch (Exception e) {
+			LOG.error("Failed to close (return) instance to pool", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisCommand.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisCommand.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisCommand.java
new file mode 100644
index 0000000..b0661c7
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisCommand.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis.common.mapper;
+
+/**
+ * All available commands for Redis. Each command belongs to a {@link RedisDataType} group.
+ */
+public enum RedisCommand {
+
+	/**
+	 * Insert the specified value at the head of the list stored at key.
+	 * If key does not exist, it is created as empty list before performing the push operations.
+	 */
+	LPUSH(RedisDataType.LIST),
+
+	/**
+	 * Insert the specified value at the tail of the list stored at key.
+	 * If key does not exist, it is created as empty list before performing the push operation.
+	 */
+	RPUSH(RedisDataType.LIST),
+
+	/**
+	 * Add the specified member to the set stored at key.
+	 * Specified member that is already a member of this set is ignored.
+	 */
+	SADD(RedisDataType.SET),
+
+	/**
+	 * Set key to hold the string value. If key already holds a value,
+	 * it is overwritten, regardless of its type.
+	 */
+	SET(RedisDataType.STRING),
+
+	/**
+	 * Adds the element to the HyperLogLog data structure stored at the variable name specified as first argument.
+	 */
+	PFADD(RedisDataType.HYPER_LOG_LOG),
+
+	/**
+	 * Posts a message to the given channel.
+	 */
+	PUBLISH(RedisDataType.PUBSUB),
+
+	/**
+	 * Adds the specified members with the specified score to the sorted set stored at key.
+	 */
+	ZADD(RedisDataType.SORTED_SET),
+
+	/**
+	 * Sets field in the hash stored at key to value. If key does not exist,
+	 * a new key holding a hash is created. If field already exists in the hash, it is overwritten.
+	 */
+	HSET(RedisDataType.HASH);
+
+	/**
+	 * The {@link RedisDataType} this command belongs to.
+	 */
+	private RedisDataType redisDataType;
+
+	RedisCommand(RedisDataType redisDataType) {
+		this.redisDataType = redisDataType;
+	}
+
+
+	/**
+	 * The {@link RedisDataType} this command belongs to.
+	 * @return the {@link RedisDataType}
+	 */
+	public RedisDataType getRedisDataType(){
+		return redisDataType;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisCommandDescription.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisCommandDescription.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisCommandDescription.java
new file mode 100644
index 0000000..1eea48a
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisCommandDescription.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis.common.mapper;
+
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/**
+ * The description of the command type. This must be passed while creating new {@link RedisMapper}.
+ * <p>When creating descriptor for the group of {@link RedisDataType#HASH} and {@link RedisDataType#SORTED_SET},
+ * you need to use first constructor {@link #RedisCommandDescription(RedisCommand, String)}.
+ * If the {@code additionalKey} is {@code null} it will throw {@code IllegalArgumentException}
+ *
+ * <p>When {@link RedisCommand} is not in group of {@link RedisDataType#HASH} and {@link RedisDataType#SORTED_SET}
+ * you can use second constructor {@link #RedisCommandDescription(RedisCommand)}
+ */
+public class RedisCommandDescription implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private RedisCommand redisCommand;
+
+	/**
+	 * This additional key is needed for the group {@link RedisDataType#HASH} and {@link RedisDataType#SORTED_SET}.
+	 * Other {@link RedisDataType} works only with two variable i.e. name of the list and value to be added.
+	 * But for {@link RedisDataType#HASH} and {@link RedisDataType#SORTED_SET} we need three variables.
+	 * <p>For {@link RedisDataType#HASH} we need hash name, hash key and element.
+	 * {@link #getAdditionalKey()} used as hash name for {@link RedisDataType#HASH}
+	 * <p>For {@link RedisDataType#SORTED_SET} we need set name, the element and it's score.
+	 * {@link #getAdditionalKey()} used as set name for {@link RedisDataType#SORTED_SET}
+	 */
+	private String additionalKey;
+
+	/**
+	 * Use this constructor when data type is {@link RedisDataType#HASH} or {@link RedisDataType#SORTED_SET}.
+	 * If different data type is specified, {@code additionalKey} is ignored.
+	 * @param redisCommand the redis command type {@link RedisCommand}
+	 * @param additionalKey additional key for Hash and Sorted set data type
+	 */
+	public RedisCommandDescription(RedisCommand redisCommand, String additionalKey) {
+		Preconditions.checkNotNull(redisCommand, "Redis command type can not be null");
+		this.redisCommand = redisCommand;
+		this.additionalKey = additionalKey;
+
+		if (redisCommand.getRedisDataType() == RedisDataType.HASH ||
+			redisCommand.getRedisDataType() == RedisDataType.SORTED_SET) {
+			if (additionalKey == null) {
+				throw new IllegalArgumentException("Hash and Sorted Set should have additional key");
+			}
+		}
+	}
+
+	/**
+	 * Use this constructor when command type is not in group {@link RedisDataType#HASH} or {@link RedisDataType#SORTED_SET}.
+	 *
+	 * @param redisCommand the redis data type {@link RedisCommand}
+	 */
+	public RedisCommandDescription(RedisCommand redisCommand) {
+		this(redisCommand, null);
+	}
+
+	/**
+	 * Returns the {@link RedisCommand}.
+	 *
+	 * @return the command type of the mapping
+	 */
+	public RedisCommand getCommand() {
+		return redisCommand;
+	}
+
+	/**
+	 * Returns the additional key if data type is {@link RedisDataType#HASH} and {@link RedisDataType#SORTED_SET}.
+	 *
+	 * @return the additional key
+	 */
+	public String getAdditionalKey() {
+		return additionalKey;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisDataType.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisDataType.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisDataType.java
new file mode 100644
index 0000000..6e3997c
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisDataType.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.redis.common.mapper;
+
+/**
+ * All available data type for Redis.
+ */
+public enum RedisDataType {
+
+	/**
+	 * Strings are the most basic kind of Redis value. Redis Strings are binary safe,
+	 * this means that a Redis string can contain any kind of data, for instance a JPEG image or a serialized Ruby object.
+	 * A String value can be at max 512 Megabytes in length.
+	 */
+	STRING,
+
+	/**
+	 * Redis Hashes are maps between string fields and string values.
+	 */
+	HASH,
+
+	/**
+	 * Redis Lists are simply lists of strings, sorted by insertion order.
+	 */
+	LIST,
+
+	/**
+	 * Redis Sets are an unordered collection of Strings.
+	 */
+	SET,
+
+	/**
+	 * Redis Sorted Sets are, similarly to Redis Sets, non repeating collections of Strings.
+	 * The difference is that every member of a Sorted Set is associated with score,
+	 * that is used in order to take the sorted set ordered, from the smallest to the greatest score.
+	 * While members are unique, scores may be repeated.
+	 */
+	SORTED_SET,
+
+	/**
+	 * HyperLogLog is a probabilistic data structure used in order to count unique things.
+	 */
+	HYPER_LOG_LOG,
+
+	/**
+	 * Redis implementation of publish and subscribe paradigm. Published messages are characterized into channels,
+	 * without knowledge of what (if any) subscribers there may be.
+	 * Subscribers express interest in one or more channels, and only receive messages
+	 * that are of interest, without knowledge of what (if any) publishers there are.
+	 */
+	PUBSUB
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisMapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisMapper.java b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisMapper.java
new file mode 100644
index 0000000..63fed19
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisMapper.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.redis.common.mapper;
+
+import org.apache.flink.api.common.functions.Function;
+
+import java.io.Serializable;
+
+/**
+ * Function that creates the description how the input data should be mapped to redis type.
+ *<p>Example:
+ *<pre>{@code
+ *private static class RedisTestMapper implements RedisMapper<Tuple2<String, String>> {
+ *    public RedisDataTypeDescription getCommandDescription() {
+ *        return new RedisDataTypeDescription(RedisCommand.PUBLISH);
+ *    }
+ *    public String getKeyFromData(Tuple2<String, String> data) {
+ *        return data.f0;
+ *    }
+ *    public String getValueFromData(Tuple2<String, String> data) {
+ *        return data.f1;
+ *    }
+ *}
+ *}</pre>
+ *
+ * @param <T> The type of the element handled by this {@code RedisMapper}
+ */
+public interface RedisMapper<T> extends Function, Serializable {
+
+	/**
+	 * Returns descriptor which defines data type.
+	 *
+	 * @return data type descriptor
+	 */
+	RedisCommandDescription getCommandDescription();
+
+	/**
+	 * Extracts key from data.
+	 *
+	 * @param data source data
+	 * @return key
+	 */
+	String getKeyFromData(T data);
+
+	/**
+	 * Extracts value from data.
+	 *
+	 * @param data source data
+	 * @return value
+	 */
+	String getValueFromData(T data);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisITCaseBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisITCaseBase.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisITCaseBase.java
new file mode 100644
index 0000000..7d98f2d
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisITCaseBase.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.redis;
+
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import redis.embedded.RedisServer;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.NetUtils.getAvailablePort;
+
+public abstract class RedisITCaseBase extends StreamingMultipleProgramsTestBase {
+
+	public static final int REDIS_PORT = getAvailablePort();
+	public static final String REDIS_HOST = "127.0.0.1";
+
+	private static RedisServer redisServer;
+
+	@BeforeClass
+	public static void createRedisServer() throws IOException, InterruptedException {
+		redisServer = new RedisServer(REDIS_PORT);
+		redisServer.start();
+	}
+
+	@AfterClass
+	public static void stopRedisServer(){
+		redisServer.stop();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSentinelClusterTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSentinelClusterTest.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSentinelClusterTest.java
new file mode 100644
index 0000000..dc59ba4
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSentinelClusterTest.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis;
+
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisSentinelConfig;
+import org.apache.flink.streaming.connectors.redis.common.container.RedisCommandsContainer;
+import org.apache.flink.streaming.connectors.redis.common.container.RedisCommandsContainerBuilder;
+import org.apache.flink.util.TestLogger;
+import org.junit.Assert;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisSentinelPool;
+import redis.embedded.RedisCluster;
+import redis.embedded.util.JedisUtil;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.apache.flink.util.NetUtils.getAvailablePort;
+
+public class RedisSentinelClusterTest extends TestLogger {
+
+	private static RedisCluster cluster;
+	private static final String REDIS_MASTER = "master";
+	private static final String TEST_KEY = "testKey";
+	private static final String TEST_VALUE = "testValue";
+	private static final List<Integer> sentinels = Arrays.asList(getAvailablePort(), getAvailablePort());
+	private static final List<Integer> group1 = Arrays.asList(getAvailablePort(), getAvailablePort());
+
+	private JedisSentinelPool jedisSentinelPool;
+	private FlinkJedisSentinelConfig jedisSentinelConfig;
+
+	@BeforeClass
+	public static void setUpCluster(){
+		cluster = RedisCluster.builder().sentinelPorts(sentinels).quorumSize(1)
+			.serverPorts(group1).replicationGroup(REDIS_MASTER, 1)
+			.build();
+		cluster.start();
+	}
+
+	@Before
+	public void setUp() {
+		Set<String> hosts = JedisUtil.sentinelHosts(cluster);
+		jedisSentinelConfig = new FlinkJedisSentinelConfig.Builder().setMasterName(REDIS_MASTER)
+			.setSentinels(hosts).build();
+		jedisSentinelPool = new JedisSentinelPool(jedisSentinelConfig.getMasterName(),
+			jedisSentinelConfig.getSentinels());
+	}
+
+	@Test
+	public void testRedisSentinelOperation() {
+		RedisCommandsContainer redisContainer = RedisCommandsContainerBuilder.build(jedisSentinelConfig);
+		Jedis jedis = null;
+		try{
+			jedis = jedisSentinelPool.getResource();
+			redisContainer.set(TEST_KEY, TEST_VALUE);
+			assertEquals(TEST_VALUE, jedis.get(TEST_KEY));
+		}finally {
+			if (jedis != null){
+				jedis.close();
+			}
+		}
+	}
+
+	@After
+	public void tearDown() throws IOException {
+		if (jedisSentinelPool != null) {
+			jedisSentinelPool.close();
+		}
+	}
+
+	@AfterClass
+	public static void tearDownCluster() throws IOException {
+		if (!cluster.isActive()) {
+			cluster.stop();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkITCase.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkITCase.java
new file mode 100644
index 0000000..21f3cca
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkITCase.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.redis;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisPoolConfig;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommandDescription;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisMapper;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import redis.clients.jedis.Jedis;
+
+import static org.junit.Assert.assertEquals;
+
+public class RedisSinkITCase extends RedisITCaseBase {
+
+	private FlinkJedisPoolConfig jedisPoolConfig;
+	private static final Long NUM_ELEMENTS = 20L;
+	private static final String REDIS_KEY = "TEST_KEY";
+	private static final String REDIS_ADDITIONAL_KEY = "TEST_ADDITIONAL_KEY";
+
+	StreamExecutionEnvironment env;
+
+
+	private Jedis jedis;
+
+	@Before
+	public void setUp(){
+		jedisPoolConfig = new FlinkJedisPoolConfig.Builder()
+			.setHost(REDIS_HOST)
+			.setPort(REDIS_PORT).build();
+		jedis = new Jedis(REDIS_HOST, REDIS_PORT);
+		env = StreamExecutionEnvironment.getExecutionEnvironment();
+	}
+
+	@Test
+	public void testRedisListDataType() throws Exception {
+		DataStreamSource<Tuple2<String, String>> source = env.addSource(new TestSourceFunction());
+		RedisSink<Tuple2<String, String>> redisSink = new RedisSink<>(jedisPoolConfig,
+			new RedisCommandMapper(RedisCommand.LPUSH));
+
+		source.addSink(redisSink);
+		env.execute("Test Redis List Data Type");
+
+		assertEquals(NUM_ELEMENTS, jedis.llen(REDIS_KEY));
+
+		jedis.del(REDIS_KEY);
+	}
+
+	@Test
+	public void testRedisSetDataType() throws Exception {
+		DataStreamSource<Tuple2<String, String>> source = env.addSource(new TestSourceFunction());
+		RedisSink<Tuple2<String, String>> redisSink = new RedisSink<>(jedisPoolConfig,
+			new RedisCommandMapper(RedisCommand.SADD));
+
+		source.addSink(redisSink);
+		env.execute("Test Redis Set Data Type");
+
+		assertEquals(NUM_ELEMENTS, jedis.scard(REDIS_KEY));
+
+		jedis.del(REDIS_KEY);
+	}
+
+	@Test
+	public void testRedisHyperLogLogDataType() throws Exception {
+		DataStreamSource<Tuple2<String, String>> source = env.addSource(new TestSourceFunction());
+		RedisSink<Tuple2<String, String>> redisSink = new RedisSink<>(jedisPoolConfig,
+			new RedisCommandMapper(RedisCommand.PFADD));
+
+		source.addSink(redisSink);
+		env.execute("Test Redis Hyper Log Log Data Type");
+
+		assertEquals(NUM_ELEMENTS, Long.valueOf(jedis.pfcount(REDIS_KEY)));
+
+		jedis.del(REDIS_KEY);
+	}
+
+	@Test
+	public void testRedisSortedSetDataType() throws Exception {
+		DataStreamSource<Tuple2<String, String>> source = env.addSource(new TestSourceFunctionSortedSet());
+		RedisSink<Tuple2<String, String>> redisSink = new RedisSink<>(jedisPoolConfig,
+			new RedisAdditionalDataMapper(RedisCommand.ZADD));
+
+		source.addSink(redisSink);
+		env.execute("Test Redis Sorted Set Data Type");
+
+		assertEquals(NUM_ELEMENTS, jedis.zcard(REDIS_ADDITIONAL_KEY));
+
+		jedis.del(REDIS_ADDITIONAL_KEY);
+	}
+
+	@Test
+	public void testRedisHashDataType() throws Exception {
+		DataStreamSource<Tuple2<String, String>> source = env.addSource(new TestSourceFunctionHash());
+		RedisSink<Tuple2<String, String>> redisSink = new RedisSink<>(jedisPoolConfig,
+			new RedisAdditionalDataMapper(RedisCommand.HSET));
+
+		source.addSink(redisSink);
+		env.execute("Test Redis Hash Data Type");
+
+		assertEquals(NUM_ELEMENTS, jedis.hlen(REDIS_ADDITIONAL_KEY));
+
+		jedis.del(REDIS_ADDITIONAL_KEY);
+	}
+
+	@After
+	public void tearDown(){
+		if(jedis != null){
+			jedis.close();
+		}
+	}
+
+	private static class TestSourceFunction implements SourceFunction<Tuple2<String, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		@Override
+		public void run(SourceContext<Tuple2<String, String>> ctx) throws Exception {
+			for (int i = 0; i < NUM_ELEMENTS && running; i++) {
+				ctx.collect(new Tuple2<>(REDIS_KEY, "message #" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+	private static class TestSourceFunctionHash implements SourceFunction<Tuple2<String, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		@Override
+		public void run(SourceContext<Tuple2<String, String>> ctx) throws Exception {
+			for (int i = 0; i < NUM_ELEMENTS && running; i++) {
+				ctx.collect(new Tuple2<>("" + i, "message #" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+	private static class TestSourceFunctionSortedSet implements SourceFunction<Tuple2<String, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		@Override
+		public void run(SourceContext<Tuple2<String, String>> ctx) throws Exception {
+			for (int i = 0; i < NUM_ELEMENTS && running; i++) {
+				ctx.collect(new Tuple2<>( "message #" + i, "" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+	public static class RedisCommandMapper implements RedisMapper<Tuple2<String, String>>{
+
+		private RedisCommand redisCommand;
+
+		public RedisCommandMapper(RedisCommand redisCommand){
+			this.redisCommand = redisCommand;
+		}
+
+		@Override
+		public RedisCommandDescription getCommandDescription() {
+			return new RedisCommandDescription(redisCommand);
+		}
+
+		@Override
+		public String getKeyFromData(Tuple2<String, String> data) {
+			return data.f0;
+		}
+
+		@Override
+		public String getValueFromData(Tuple2<String, String> data) {
+			return data.f1;
+		}
+	}
+
+	public static class RedisAdditionalDataMapper implements RedisMapper<Tuple2<String, String>>{
+
+		private RedisCommand redisCommand;
+
+		public RedisAdditionalDataMapper(RedisCommand redisCommand){
+			this.redisCommand = redisCommand;
+		}
+
+		@Override
+		public RedisCommandDescription getCommandDescription() {
+			return new RedisCommandDescription(redisCommand, REDIS_ADDITIONAL_KEY);
+		}
+
+		@Override
+		public String getKeyFromData(Tuple2<String, String> data) {
+			return data.f0;
+		}
+
+		@Override
+		public String getValueFromData(Tuple2<String, String> data) {
+			return data.f1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkPublishITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkPublishITCase.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkPublishITCase.java
new file mode 100644
index 0000000..caf3945
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkPublishITCase.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisPoolConfig;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommandDescription;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisMapper;
+
+import redis.clients.jedis.JedisPool;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+import redis.clients.jedis.JedisPubSub;
+
+import static org.junit.Assert.assertEquals;
+
+public class RedisSinkPublishITCase extends RedisITCaseBase {
+
+	private static final int NUM_ELEMENTS = 20;
+	private static final String REDIS_CHANNEL = "CHANNEL";
+
+	private static final List<String> sourceList = new ArrayList<>();
+	private Thread sinkThread;
+	private PubSub pubSub;
+
+	@Before
+	public void before() throws Exception {
+		pubSub = new PubSub();
+		sinkThread = new Thread(new Subscribe(pubSub));
+	}
+
+	@Test
+	public void redisSinkTest() throws Exception {
+		sinkThread.start();
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		FlinkJedisPoolConfig jedisPoolConfig = new FlinkJedisPoolConfig.Builder()
+			.setHost(REDIS_HOST)
+			.setPort(REDIS_PORT).build();
+		DataStreamSource<Tuple2<String, String>> source = env.addSource(new TestSourceFunction());
+
+		RedisSink<Tuple2<String, String>> redisSink = new RedisSink<>(jedisPoolConfig, new RedisTestMapper());
+
+		source.addSink(redisSink);
+
+		env.execute("Redis Sink Test");
+
+		assertEquals(NUM_ELEMENTS, sourceList.size());
+	}
+
+	@After
+	public void after() throws Exception {
+		pubSub.unsubscribe();
+		sinkThread.join();
+		sourceList.clear();
+	}
+
+	private class Subscribe implements Runnable {
+		private PubSub localPubSub;
+		private Subscribe(PubSub pubSub){
+			this.localPubSub = pubSub;
+		}
+
+		@Override
+		public void run() {
+			JedisPool pool = new JedisPool(REDIS_HOST, REDIS_PORT);
+			pool.getResource().subscribe(localPubSub, REDIS_CHANNEL);
+		}
+	}
+
+	private static class TestSourceFunction implements SourceFunction<Tuple2<String, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		@Override
+		public void run(SourceContext<Tuple2<String, String>> ctx) throws Exception {
+			for (int i = 0; i < NUM_ELEMENTS && running; i++) {
+				ctx.collect(new Tuple2<>(REDIS_CHANNEL, "message #" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+	public static class PubSub extends JedisPubSub {
+
+		@Override
+		public void onMessage(String channel, String message) {
+			sourceList.add(message);
+		}
+
+	}
+
+	private static class RedisTestMapper implements RedisMapper<Tuple2<String, String>>{
+
+		@Override
+		public RedisCommandDescription getCommandDescription() {
+			return new RedisCommandDescription(RedisCommand.PUBLISH);
+		}
+
+		@Override
+		public String getKeyFromData(Tuple2<String, String> data) {
+			return data.f0;
+		}
+
+		@Override
+		public String getValueFromData(Tuple2<String, String> data) {
+			return data.f1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkTest.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkTest.java
new file mode 100644
index 0000000..59f59f2
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisSinkTest.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.redis;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisClusterConfig;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisConfigBase;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisPoolConfig;
+import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisSentinelConfig;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommandDescription;
+import org.apache.flink.streaming.connectors.redis.common.mapper.RedisMapper;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+import redis.clients.jedis.exceptions.JedisConnectionException;
+
+import java.net.InetSocketAddress;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.fail;
+
+public class RedisSinkTest extends TestLogger {
+
+	@Test(expected=NullPointerException.class)
+	public void shouldThrowNullPointExceptionIfDataMapperIsNull(){
+		new RedisSink<>(new FlinkJedisClusterConfig.Builder().build(), null);
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void shouldThrowNullPointerExceptionIfCommandDescriptionIsNull(){
+		new RedisSink<>(new FlinkJedisClusterConfig.Builder().build(), new TestMapper(null));
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void shouldThrowNullPointerExceptionIfConfigurationIsNull(){
+		new RedisSink<>(null, new TestMapper(new RedisCommandDescription(RedisCommand.LPUSH)));
+	}
+
+	@Test
+	public void testRedisDownBehavior() throws Exception {
+
+		// create a wrong configuration so that open() fails.
+
+		FlinkJedisPoolConfig wrongJedisPoolConfig = new FlinkJedisPoolConfig.Builder()
+			.setHost("127.0.0.1")
+			.setPort(1234).build();
+
+		testDownBehavior(wrongJedisPoolConfig);
+	}
+
+	@Test
+	public void testRedisClusterDownBehavior() throws Exception {
+
+		Set<InetSocketAddress> hosts = new HashSet<>();
+		hosts.add(new InetSocketAddress("127.0.0.1", 1234));
+
+		// create a wrong configuration so that open() fails.
+
+		FlinkJedisClusterConfig wrongJedisClusterConfig = new FlinkJedisClusterConfig.Builder()
+			.setNodes(hosts)
+			.setTimeout(100)
+			.setMaxIdle(1)
+			.setMaxTotal(1)
+			.setMinIdle(1).build();
+
+		testDownBehavior(wrongJedisClusterConfig);
+	}
+
+	@Test
+	public void testRedisSentinelDownBehavior() throws Exception {
+
+		Set<String> hosts = new HashSet<>();
+		hosts.add("localhost:55095");
+
+		// create a wrong configuration so that open() fails.
+
+		FlinkJedisSentinelConfig wrongJedisSentinelConfig = new FlinkJedisSentinelConfig.Builder()
+			.setMasterName("master")
+			.setSentinels(hosts)
+			.build();
+
+		testDownBehavior(wrongJedisSentinelConfig);
+	}
+
+	private void testDownBehavior(FlinkJedisConfigBase config) throws Exception {
+		RedisSink<Tuple2<String, String>> redisSink = new RedisSink<>(config,
+			new RedisSinkITCase.RedisCommandMapper(RedisCommand.SADD));
+
+		try {
+			redisSink.open(new Configuration());
+		} catch (Exception e) {
+
+			// search for nested JedisConnectionExceptions
+			// because this is the expected behavior
+
+			Throwable t = e;
+			int depth = 0;
+			while (!(t instanceof JedisConnectionException)) {
+				t = t.getCause();
+				if (t == null || depth++ == 20) {
+					throw e;
+				}
+			}
+		}
+	}
+
+	private class TestMapper implements RedisMapper<Tuple2<String, String>>{
+		private RedisCommandDescription redisCommandDescription;
+
+		public TestMapper(RedisCommandDescription redisCommandDescription){
+			this.redisCommandDescription = redisCommandDescription;
+		}
+		@Override
+		public RedisCommandDescription getCommandDescription() {
+			return redisCommandDescription;
+		}
+
+		@Override
+		public String getKeyFromData(Tuple2<String, String> data) {
+			return data.f0;
+		}
+
+		@Override
+		public String getValueFromData(Tuple2<String, String> data) {
+			return data.f1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBaseTest.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBaseTest.java
new file mode 100644
index 0000000..ed1d713
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBaseTest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.redis.common.config;
+
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+public class FlinkJedisConfigBaseTest extends TestLogger {
+
+	@Test(expected = IllegalArgumentException.class)
+	public void shouldThrowIllegalArgumentExceptionIfTimeOutIsNegative(){
+		new TestConfig(-1, 0, 0, 0);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void shouldThrowIllegalArgumentExceptionIfMaxTotalIsNegative(){
+		new TestConfig(1, -1, 0, 0);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void shouldThrowIllegalArgumentExceptionIfMaxIdleIsNegative(){
+		new TestConfig(0, 0, -1, 0);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void shouldThrowIllegalArgumentExceptionIfMinIdleIsNegative(){
+		new TestConfig(0, 0, 0, -1);
+	}
+
+	private class TestConfig extends FlinkJedisConfigBase{
+
+		protected TestConfig(int connectionTimeout, int maxTotal, int maxIdle, int minIdle) {
+			super(connectionTimeout, maxTotal, maxIdle, minIdle);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisClusterConfigTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisClusterConfigTest.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisClusterConfigTest.java
new file mode 100644
index 0000000..40db578
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisClusterConfigTest.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.redis.common.config;
+
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import java.net.InetSocketAddress;
+import java.util.HashSet;
+import java.util.Set;
+
+public class JedisClusterConfigTest extends TestLogger {
+
+	@Test(expected = NullPointerException.class)
+	public void shouldThrowNullPointExceptionIfNodeValueIsNull(){
+		FlinkJedisClusterConfig.Builder builder = new FlinkJedisClusterConfig.Builder();
+		builder.setMinIdle(0)
+			.setMaxIdle(0)
+			.setMaxTotal(0)
+			.setTimeout(0)
+			.build();
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void shouldThrowIllegalArgumentExceptionIfNodeValuesAreEmpty(){
+		Set<InetSocketAddress> set = new HashSet<>();
+		FlinkJedisClusterConfig.Builder builder = new FlinkJedisClusterConfig.Builder();
+		builder.setMinIdle(0)
+			.setMaxIdle(0)
+			.setMaxTotal(0)
+			.setTimeout(0)
+			.setNodes(set)
+			.build();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisPoolConfigTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisPoolConfigTest.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisPoolConfigTest.java
new file mode 100644
index 0000000..dc16cfe
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisPoolConfigTest.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.redis.common.config;
+
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+public class JedisPoolConfigTest extends TestLogger {
+
+	@Test(expected = NullPointerException.class)
+	public void shouldThrowNullPointExceptionIfHostValueIsNull(){
+		FlinkJedisPoolConfig.Builder builder = new FlinkJedisPoolConfig.Builder();
+		builder.build();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisSentinelConfigTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisSentinelConfigTest.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisSentinelConfigTest.java
new file mode 100644
index 0000000..8445fae
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisSentinelConfigTest.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.redis.common.config;
+
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class JedisSentinelConfigTest extends TestLogger {
+
+	public static final String MASTER_NAME = "test-master";
+
+	@Test(expected = NullPointerException.class)
+	public void shouldThrowNullPointExceptionIfMasterValueIsNull(){
+		FlinkJedisSentinelConfig.Builder builder = new FlinkJedisSentinelConfig.Builder();
+		Set<String> sentinels = new HashSet<>();
+		sentinels.add("127.0.0.1");
+		builder.setSentinels(sentinels).build();
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void shouldThrowNullPointExceptionIfSentinelsValueIsNull(){
+		FlinkJedisSentinelConfig.Builder builder = new FlinkJedisSentinelConfig.Builder();
+		builder.setMasterName(MASTER_NAME).build();
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void shouldThrowNullPointExceptionIfSentinelsValueIsEmpty(){
+		FlinkJedisSentinelConfig.Builder builder = new FlinkJedisSentinelConfig.Builder();
+		Set<String> sentinels = new HashSet<>();
+		builder.setMasterName(MASTER_NAME).setSentinels(sentinels).build();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisDataTypeDescriptionTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisDataTypeDescriptionTest.java b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisDataTypeDescriptionTest.java
new file mode 100644
index 0000000..b0eee48
--- /dev/null
+++ b/flink-connectors/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisDataTypeDescriptionTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.redis.common.mapper;
+
+import org.apache.flink.streaming.connectors.redis.RedisSinkITCase;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class RedisDataTypeDescriptionTest extends TestLogger {
+
+	@Test(expected=IllegalArgumentException.class)
+	public void shouldThrowExceptionIfAdditionalKeyIsNotGivenForHashDataType(){
+		RedisSinkITCase.RedisCommandMapper redisCommandMapper = new RedisSinkITCase.RedisCommandMapper(RedisCommand.HSET);
+		redisCommandMapper.getCommandDescription();
+	}
+
+	@Test
+	public void shouldReturnNullForAdditionalDataType(){
+		RedisSinkITCase.RedisCommandMapper redisCommandMapper = new RedisSinkITCase.RedisCommandMapper(RedisCommand.LPUSH);
+		RedisCommandDescription redisDataTypeDescription = redisCommandMapper.getCommandDescription();
+		assertEquals(RedisDataType.LIST, redisDataTypeDescription.getCommand().getRedisDataType());
+		assertNull(redisDataTypeDescription.getAdditionalKey());
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
new file mode 100644
index 0000000..66fa237
--- /dev/null
+++ b/flink-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.twitter;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.Properties;
+
+import com.twitter.hbc.common.DelimitedStreamReader;
+import com.twitter.hbc.core.endpoint.StreamingEndpoint;
+import com.twitter.hbc.core.processor.HosebirdMessageProcessor;
+import org.apache.flink.api.common.functions.StoppableFunction;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.twitter.hbc.ClientBuilder;
+import com.twitter.hbc.core.Constants;
+import com.twitter.hbc.core.endpoint.StatusesSampleEndpoint;
+import com.twitter.hbc.httpclient.BasicClient;
+import com.twitter.hbc.httpclient.auth.Authentication;
+import com.twitter.hbc.httpclient.auth.OAuth1;
+
+/**
+ * Implementation of {@link SourceFunction} specialized to emit tweets from
+ * Twitter. This is not a parallel source because the Twitter API only allows
+ * two concurrent connections.
+ */
+public class TwitterSource extends RichSourceFunction<String> implements StoppableFunction {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TwitterSource.class);
+
+	private static final long serialVersionUID = 1L;
+
+	// ----- Required property keys
+
+	public static final String CONSUMER_KEY = "twitter-source.consumerKey";
+
+	public static final String CONSUMER_SECRET = "twitter-source.consumerSecret";
+
+	public static final String TOKEN = "twitter-source.token";
+
+	public static final String TOKEN_SECRET = "twitter-source.tokenSecret";
+
+	// ------ Optional property keys
+
+	public static final String CLIENT_NAME = "twitter-source.name";
+
+	public static final String CLIENT_HOSTS = "twitter-source.hosts";
+
+	public static final String CLIENT_BUFFER_SIZE = "twitter-source.bufferSize";
+
+	// ----- Fields set by the constructor
+
+	private final Properties properties;
+
+	private EndpointInitializer initializer = new SampleStatusesEndpoint();
+
+	// ----- Runtime fields
+	private transient BasicClient client;
+	private transient Object waitLock;
+	private transient boolean running = true;
+
+
+	/**
+	 * Create {@link TwitterSource} for streaming
+	 * 
+	 * @param properties For the source
+	 */
+	public TwitterSource(Properties properties) {
+		checkProperty(properties, CONSUMER_KEY);
+		checkProperty(properties, CONSUMER_SECRET);
+		checkProperty(properties, TOKEN);
+		checkProperty(properties, TOKEN_SECRET);
+
+		this.properties = properties;
+	}
+
+	private static void checkProperty(Properties p, String key) {
+		if(!p.containsKey(key)) {
+			throw new IllegalArgumentException("Required property '" + key + "' not set.");
+		}
+	}
+
+
+	/**
+	 * Set a custom endpoint initializer.
+	 */
+	public void setCustomEndpointInitializer(EndpointInitializer initializer) {
+		Objects.requireNonNull(initializer, "Initializer has to be set");
+		ClosureCleaner.ensureSerializable(initializer);
+		this.initializer = initializer;
+	}
+
+	// ----- Source lifecycle
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		waitLock = new Object();
+	}
+
+
+	@Override
+	public void run(final SourceContext<String> ctx) throws Exception {
+		LOG.info("Initializing Twitter Streaming API connection");
+
+		StreamingEndpoint endpoint = initializer.createEndpoint();
+
+		Authentication auth = new OAuth1(properties.getProperty(CONSUMER_KEY),
+			properties.getProperty(CONSUMER_SECRET),
+			properties.getProperty(TOKEN),
+			properties.getProperty(TOKEN_SECRET));
+
+		client = new ClientBuilder()
+			.name(properties.getProperty(CLIENT_NAME, "flink-twitter-source"))
+			.hosts(properties.getProperty(CLIENT_HOSTS, Constants.STREAM_HOST))
+			.endpoint(endpoint)
+			.authentication(auth)
+			.processor(new HosebirdMessageProcessor() {
+				public DelimitedStreamReader reader;
+
+				@Override
+				public void setup(InputStream input) {
+					reader = new DelimitedStreamReader(input, Constants.DEFAULT_CHARSET, Integer.parseInt(properties.getProperty(CLIENT_BUFFER_SIZE, "50000")));
+				}
+
+				@Override
+				public boolean process() throws IOException, InterruptedException {
+					String line = reader.readLine();
+					ctx.collect(line);
+					return true;
+				}
+			})
+			.build();
+
+		client.connect();
+		running = true;
+
+		LOG.info("Twitter Streaming API connection established successfully");
+
+		// just wait now
+		while(running) {
+			synchronized (waitLock) {
+				waitLock.wait(100L);
+			}
+		}
+	}
+
+	@Override
+	public void close() {
+		this.running = false;
+		LOG.info("Closing source");
+		if (client != null) {
+			// client seems to be thread-safe
+			client.stop();
+		}
+		// leave main method
+		synchronized (waitLock) {
+			waitLock.notify();
+		}
+	}
+
+	@Override
+	public void cancel() {
+		LOG.info("Cancelling Twitter source");
+		close();
+	}
+
+	@Override
+	public void stop() {
+		LOG.info("Stopping Twitter source");
+		close();
+	}
+
+	// ------ Custom endpoints
+
+	/**
+	 * Implementing this interface allows users of this source to set a custom endpoint.
+	 */
+	public interface EndpointInitializer {
+		StreamingEndpoint createEndpoint();
+	}
+
+	/**
+	 * Default endpoint initializer returning the {@see StatusesSampleEndpoint}.
+	 */
+	private static class SampleStatusesEndpoint implements EndpointInitializer, Serializable {
+		@Override
+		public StreamingEndpoint createEndpoint() {
+			// this default endpoint initializer returns the sample endpoint: Returning a sample from the firehose (all tweets)
+			StatusesSampleEndpoint endpoint = new StatusesSampleEndpoint();
+			endpoint.stallWarnings(false);
+			endpoint.delimited(false);
+			return endpoint;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/pom.xml b/flink-connectors/flink-hadoop-compatibility/pom.xml
new file mode 100644
index 0000000..5938560
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/pom.xml
@@ -0,0 +1,182 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+	
+	<modelVersion>4.0.0</modelVersion>
+	
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+	<name>flink-hadoop-compatibility</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+
+		<!-- core dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-scala_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-hadoop2</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+		
+	</dependencies>
+
+
+	<build>
+		<plugins>
+			<!-- activate API compatibility checks -->
+			<plugin>
+				<groupId>com.github.siom79.japicmp</groupId>
+				<artifactId>japicmp-maven-plugin</artifactId>
+			</plugin>
+			<!-- Scala Compiler -->
+			<plugin>
+				<groupId>net.alchim31.maven</groupId>
+				<artifactId>scala-maven-plugin</artifactId>
+				<version>3.1.4</version>
+				<executions>
+					<!-- Run scala compiler in the process-resources phase, so that dependencies on
+						scala classes can be resolved later in the (Java) compile phase -->
+					<execution>
+						<id>scala-compile-first</id>
+						<phase>process-resources</phase>
+						<goals>
+							<goal>compile</goal>
+						</goals>
+					</execution>
+				</executions>
+				<configuration>
+					<jvmArgs>
+						<jvmArg>-Xms128m</jvmArg>
+						<jvmArg>-Xmx512m</jvmArg>
+					</jvmArgs>
+				</configuration>
+			</plugin>
+
+			<!-- Eclipse Integration -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-eclipse-plugin</artifactId>
+				<version>2.8</version>
+				<configuration>
+					<downloadSources>true</downloadSources>
+					<projectnatures>
+						<projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
+						<projectnature>org.eclipse.jdt.core.javanature</projectnature>
+					</projectnatures>
+					<buildcommands>
+						<buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
+					</buildcommands>
+					<classpathContainers>
+						<classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
+						<classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
+					</classpathContainers>
+					<excludes>
+						<exclude>org.scala-lang:scala-library</exclude>
+						<exclude>org.scala-lang:scala-compiler</exclude>
+					</excludes>
+					<sourceIncludes>
+						<sourceInclude>**/*.scala</sourceInclude>
+						<sourceInclude>**/*.java</sourceInclude>
+					</sourceIncludes>
+				</configuration>
+			</plugin>
+
+			<!-- Adding scala source directories to build path -->
+			<plugin>
+				<groupId>org.codehaus.mojo</groupId>
+				<artifactId>build-helper-maven-plugin</artifactId>
+				<version>1.7</version>
+				<executions>
+					<!-- Add src/main/scala to eclipse build path -->
+					<execution>
+						<id>add-source</id>
+						<phase>generate-sources</phase>
+						<goals>
+							<goal>add-source</goal>
+						</goals>
+						<configuration>
+							<sources>
+								<source>src/main/scala</source>
+							</sources>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+
+			<!-- Scala Code Style, most of the configuration done via plugin management -->
+			<plugin>
+				<groupId>org.scalastyle</groupId>
+				<artifactId>scalastyle-maven-plugin</artifactId>
+				<configuration>
+					<configLocation>${project.basedir}/../../tools/maven/scalastyle-config.xml</configLocation>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
new file mode 100644
index 0000000..7bcb4bf
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.typeinfo.AtomicType;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.runtime.WritableComparator;
+import org.apache.flink.api.java.typeutils.runtime.WritableSerializer;
+import org.apache.hadoop.io.Writable;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Type information for data types that extend Hadoop's {@link Writable} interface. The Writable
+ * interface defines the serialization and deserialization routines for the data type.
+ *
+ * @param <T> The type of the class represented by this type information.
+ */
+@Public
+public class WritableTypeInfo<T extends Writable> extends TypeInformation<T> implements AtomicType<T> {
+	
+	private static final long serialVersionUID = 1L;
+	
+	private final Class<T> typeClass;
+
+	@PublicEvolving
+	public WritableTypeInfo(Class<T> typeClass) {
+		this.typeClass = checkNotNull(typeClass);
+
+		checkArgument(
+			Writable.class.isAssignableFrom(typeClass) && !typeClass.equals(Writable.class),
+			"WritableTypeInfo can only be used for subclasses of %s", Writable.class.getName());
+	}
+
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	@Override
+	@PublicEvolving
+	public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) {
+		if(Comparable.class.isAssignableFrom(typeClass)) {
+			return new WritableComparator(sortOrderAscending, typeClass);
+		}
+		else {
+			throw new UnsupportedOperationException("Cannot create Comparator for "+typeClass.getCanonicalName()+". " +
+													"Class does not implement Comparable interface.");
+		}
+	}
+
+	@Override
+	@PublicEvolving
+	public boolean isBasicType() {
+		return false;
+	}
+
+	@Override
+	@PublicEvolving
+	public boolean isTupleType() {
+		return false;
+	}
+
+	@Override
+	@PublicEvolving
+	public int getArity() {
+		return 1;
+	}
+	
+	@Override
+	@PublicEvolving
+	public int getTotalFields() {
+		return 1;
+	}
+
+	@Override
+	@PublicEvolving
+	public Class<T> getTypeClass() {
+		return this.typeClass;
+	}
+
+	@Override
+	@PublicEvolving
+	public boolean isKeyType() {
+		return Comparable.class.isAssignableFrom(typeClass);
+	}
+
+	@Override
+	@PublicEvolving
+	public TypeSerializer<T> createSerializer(ExecutionConfig executionConfig) {
+		return new WritableSerializer<T>(typeClass);
+	}
+	
+	@Override
+	public String toString() {
+		return "WritableType<" + typeClass.getName() + ">";
+	}	
+	
+	@Override
+	public int hashCode() {
+		return typeClass.hashCode();
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof WritableTypeInfo) {
+			@SuppressWarnings("unchecked")
+			WritableTypeInfo<T> writableTypeInfo = (WritableTypeInfo<T>) obj;
+
+			return writableTypeInfo.canEqual(this) &&
+				typeClass == writableTypeInfo.typeClass;
+
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof WritableTypeInfo;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+
+	@PublicEvolving
+	static <T extends Writable> TypeInformation<T> getWritableTypeInfo(Class<T> typeClass) {
+		if (Writable.class.isAssignableFrom(typeClass) && !typeClass.equals(Writable.class)) {
+			return new WritableTypeInfo<T>(typeClass);
+		}
+		else {
+			throw new InvalidTypesException("The given class is no subclass of " + Writable.class.getName());
+		}
+	}
+	
+}


[40/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
new file mode 100644
index 0000000..cf2c373
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
@@ -0,0 +1,1082 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs.bucketing;
+
+import org.apache.commons.lang3.time.StopWatch;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.connectors.fs.Clock;
+import org.apache.flink.streaming.connectors.fs.SequenceFileWriter;
+import org.apache.flink.streaming.connectors.fs.StringWriter;
+import org.apache.flink.streaming.connectors.fs.Writer;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.Preconditions;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.Iterator;
+
+/**
+ * Sink that emits its input elements to {@link org.apache.hadoop.fs.FileSystem} files within
+ * buckets. This is integrated with the checkpointing mechanism to provide exactly once semantics.
+ *
+ * <p>
+ * When creating the sink a {@code basePath} must be specified. The base directory contains
+ * one directory for every bucket. The bucket directories themselves contain several part files,
+ * one for each parallel subtask of the sink. These part files contain the actual output data.
+ *
+ * <p>
+ * The sink uses a {@link Bucketer} to determine in which bucket directory each element should
+ * be written to inside the base directory. The {@code Bucketer} can, for example, use time or
+ * a property of the element to determine the bucket directory. The default {@code Bucketer} is a
+ * {@link DateTimeBucketer} which will create one new bucket every hour. You can specify
+ * a custom {@code Bucketer} using {@link #setBucketer(Bucketer)}. For example, use the
+ * {@link BasePathBucketer} if you don't want to have buckets but still want to write part-files
+ * in a fault-tolerant way.
+ *
+ * <p>
+ * The filenames of the part files contain the part prefix, the parallel subtask index of the sink
+ * and a rolling counter. For example the file {@code "part-1-17"} contains the data from
+ * {@code subtask 1} of the sink and is the {@code 17th} bucket created by that subtask. Per default
+ * the part prefix is {@code "part"} but this can be configured using {@link #setPartPrefix(String)}.
+ * When a part file becomes bigger than the user-specified batch size the current part file is closed,
+ * the part counter is increased and a new part file is created. The batch size defaults to {@code 384MB},
+ * this can be configured using {@link #setBatchSize(long)}.
+ *
+ * <p>
+ * In some scenarios, the open buckets are required to change based on time. In these cases, the sink
+ * needs to determine when a bucket has become inactive, in order to flush and close the part file.
+ * To support this there are two configurable settings:
+ * <ol>
+ *     <li>the frequency to check for inactive buckets, configured by {@link #setInactiveBucketCheckInterval(long)},
+ *     and</li>
+ *     <li>the minimum amount of time a bucket has to not receive any data before it is considered inactive,
+ *     configured by {@link #setInactiveBucketThreshold(long)}</li>
+ * </ol>
+ * Both of these parameters default to {@code 60,000 ms}, or {@code 1 min}.
+ *
+ * <p>
+ * Part files can be in one of three states: {@code in-progress}, {@code pending} or {@code finished}.
+ * The reason for this is how the sink works together with the checkpointing mechanism to provide exactly-once
+ * semantics and fault-tolerance. The part file that is currently being written to is {@code in-progress}. Once
+ * a part file is closed for writing it becomes {@code pending}. When a checkpoint is successful the currently
+ * pending files will be moved to {@code finished}.
+ *
+ * <p>
+ * If case of a failure, and in order to guarantee exactly-once semantics, the sink should roll back to the state it
+ * had when that last successful checkpoint occurred. To this end, when restoring, the restored files in {@code pending}
+ * state are transferred into the {@code finished} state while any {@code in-progress} files are rolled back, so that
+ * they do not contain data that arrived after the checkpoint from which we restore. If the {@code FileSystem} supports
+ * the {@code truncate()} method this will be used to reset the file back to its previous state. If not, a special
+ * file with the same name as the part file and the suffix {@code ".valid-length"} will be created that contains the
+ * length up to which the file contains valid data. When reading the file, it must be ensured that it is only read up
+ * to that point. The prefixes and suffixes for the different file states and valid-length files can be configured
+ * using the adequate setter method, e.g. {@link #setPendingSuffix(String)}.
+ *
+ * <p>
+ * <b>NOTE:</b>
+ * <ol>
+ *     <li>
+ *         If checkpointing is not enabled the pending files will never be moved to the finished state. In that case,
+ *         the pending suffix/prefix can be set to {@code ""} to make the sink work in a non-fault-tolerant way but
+ *         still provide output without prefixes and suffixes.
+ *     </li>
+ *     <li>
+ *         The part files are written using an instance of {@link Writer}. By default, a
+ *         {@link org.apache.flink.streaming.connectors.fs.StringWriter} is used, which writes the result
+ *         of {@code toString()} for every element, separated by newlines. You can configure the writer using  the
+ *         {@link #setWriter(Writer)}. For example, {@link org.apache.flink.streaming.connectors.fs.SequenceFileWriter}
+ *         can be used to write Hadoop {@code SequenceFiles}.
+ *     </li>
+ * </ol>
+ *
+ * <p>
+ * Example:
+ * <pre>{@code
+ *     new BucketingSink<Tuple2<IntWritable, Text>>(outPath)
+ *         .setWriter(new SequenceFileWriter<IntWritable, Text>())
+ *         .setBucketer(new DateTimeBucketer("yyyy-MM-dd--HHmm")
+ * }</pre>
+ *
+ * This will create a sink that writes to {@code SequenceFiles} and rolls every minute.
+ *
+ * @see DateTimeBucketer
+ * @see StringWriter
+ * @see SequenceFileWriter
+ *
+ * @param <T> Type of the elements emitted by this sink
+ */
+public class BucketingSink<T>
+		extends RichSinkFunction<T>
+		implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener, ProcessingTimeCallback {
+
+	private static final long serialVersionUID = 1L;
+
+	private static Logger LOG = LoggerFactory.getLogger(BucketingSink.class);
+
+	// --------------------------------------------------------------------------------------------
+	//  User configuration values
+	// --------------------------------------------------------------------------------------------
+	// These are initialized with some defaults but are meant to be changeable by the user
+
+	/**
+	 * The default maximum size of part files (currently {@code 384 MB}).
+	 */
+	private final long DEFAULT_BATCH_SIZE = 1024L * 1024L * 384L;
+
+	/**
+	 * The default time between checks for inactive buckets. By default, {60 sec}.
+	 */
+	private final long DEFAULT_INACTIVE_BUCKET_CHECK_INTERVAL_MS = 60 * 1000L;
+
+	/**
+	 * The default threshold (in {@code ms}) for marking a bucket as inactive and
+	 * closing its part files. By default, {60 sec}.
+	 */
+	private final long DEFAULT_INACTIVE_BUCKET_THRESHOLD_MS = 60 * 1000L;
+
+	/**
+	 * The suffix for {@code in-progress} part files. These are files we are
+	 * currently writing to, but which were not yet confirmed by a checkpoint.
+	 */
+	private final String DEFAULT_IN_PROGRESS_SUFFIX = ".in-progress";
+
+	/**
+	 * The prefix for {@code in-progress} part files. These are files we are
+	 * currently writing to, but which were not yet confirmed by a checkpoint.
+	 */
+	private final String DEFAULT_IN_PROGRESS_PREFIX = "_";
+
+	/**
+	 * The suffix for {@code pending} part files. These are closed files that we are
+	 * not currently writing to (inactive or reached {@link #batchSize}), but which
+	 * were not yet confirmed by a checkpoint.
+	 */
+	private final String DEFAULT_PENDING_SUFFIX = ".pending";
+
+	/**
+	 * The prefix for {@code pending} part files. These are closed files that we are
+	 * not currently writing to (inactive or reached {@link #batchSize}), but which
+	 * were not yet confirmed by a checkpoint.
+	 */
+	private final String DEFAULT_PENDING_PREFIX = "_";
+
+	/**
+	 * When {@code truncate()} is not supported by the used {@link FileSystem}, we create
+	 * a file along the part file with this suffix that contains the length up to which
+	 * the part file is valid.
+	 */
+	private final String DEFAULT_VALID_SUFFIX = ".valid-length";
+
+	/**
+	 * When {@code truncate()} is not supported by the used {@link FileSystem}, we create
+	 * a file along the part file with this preffix that contains the length up to which
+	 * the part file is valid.
+	 */
+	private final String DEFAULT_VALID_PREFIX = "_";
+
+	/**
+	 * The default prefix for part files.
+	 */
+	private final String DEFAULT_PART_REFIX = "part";
+
+	/**
+	 * The default timeout for asynchronous operations such as recoverLease and truncate (in {@code ms}).
+	 */
+	private final long DEFAULT_ASYNC_TIMEOUT_MS = 60 * 1000;
+
+
+	/**
+	 * The base {@code Path} that stores all bucket directories.
+	 */
+	private final String basePath;
+
+	/**
+	 * The {@code Bucketer} that is used to determine the path of bucket directories.
+	 */
+	private Bucketer<T> bucketer;
+
+	/**
+	 * We have a template and call duplicate() for each parallel writer in open() to get the actual
+	 * writer that is used for the part files.
+	 */
+	private Writer<T> writerTemplate;
+
+	private long batchSize = DEFAULT_BATCH_SIZE;
+	private long inactiveBucketCheckInterval = DEFAULT_INACTIVE_BUCKET_CHECK_INTERVAL_MS;
+	private long inactiveBucketThreshold = DEFAULT_INACTIVE_BUCKET_THRESHOLD_MS;
+
+	// These are the actually configured prefixes/suffixes
+	private String inProgressSuffix = DEFAULT_IN_PROGRESS_SUFFIX;
+	private String inProgressPrefix = DEFAULT_IN_PROGRESS_PREFIX;
+
+	private String pendingSuffix = DEFAULT_PENDING_SUFFIX;
+	private String pendingPrefix = DEFAULT_PENDING_PREFIX;
+
+	private String validLengthSuffix = DEFAULT_VALID_SUFFIX;
+	private String validLengthPrefix= DEFAULT_VALID_PREFIX;
+
+	private String partPrefix = DEFAULT_PART_REFIX;
+
+	/**
+	 * The timeout for asynchronous operations such as recoverLease and truncate (in {@code ms}).
+	 */
+	private long asyncTimeout = DEFAULT_ASYNC_TIMEOUT_MS;
+
+	// --------------------------------------------------------------------------------------------
+	//  Internal fields (not configurable by user)
+	// -------------------------------------------�-------------------------------------------------
+
+	/**
+	 * We use reflection to get the .truncate() method, this is only available starting with Hadoop 2.7
+	 */
+	private transient Method refTruncate;
+
+	/**
+	 * The state object that is handled by Flink from snapshot/restore. This contains state for
+	 * every open bucket: the current in-progress part file path, its valid length and the pending part files.
+	 */
+	private transient State<T> state;
+
+	private transient ListState<State<T>> restoredBucketStates;
+
+	/**
+	 * User-defined FileSystem parameters
+	 */
+	private Configuration fsConfig;
+
+	/**
+	 * The FileSystem reference.
+	 */
+	private transient FileSystem fs;
+
+	private transient Clock clock;
+
+	private transient ProcessingTimeService processingTimeService;
+
+	/**
+	 * Creates a new {@code BucketingSink} that writes files to the given base directory.
+	 *
+	 * <p>
+	 * This uses a{@link DateTimeBucketer} as {@link Bucketer} and a {@link StringWriter} has writer.
+	 * The maximum bucket size is set to 384 MB.
+	 *
+	 * @param basePath The directory to which to write the bucket files.
+	 */
+	public BucketingSink(String basePath) {
+		this.basePath = basePath;
+		this.bucketer = new DateTimeBucketer<>();
+		this.writerTemplate = new StringWriter<>();
+	}
+
+	/**
+	 * Specify a custom {@code Configuration} that will be used when creating
+	 * the {@link FileSystem} for writing.
+	 */
+	public BucketingSink<T> setFSConfig(Configuration config) {
+		this.fsConfig = new Configuration();
+		fsConfig.addAll(config);
+		return this;
+	}
+
+	/**
+	 * Specify a custom {@code Configuration} that will be used when creating
+	 * the {@link FileSystem} for writing.
+	 */
+	public BucketingSink<T> setFSConfig(org.apache.hadoop.conf.Configuration config) {
+		this.fsConfig = new Configuration();
+		for(Map.Entry<String, String> entry : config) {
+			fsConfig.setString(entry.getKey(), entry.getValue());
+		}
+		return this;
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
+		if (this.writerTemplate instanceof InputTypeConfigurable) {
+			((InputTypeConfigurable) writerTemplate).setInputType(type, executionConfig);
+		}
+	}
+
+	@Override
+	public void initializeState(FunctionInitializationContext context) throws Exception {
+		Preconditions.checkArgument(this.restoredBucketStates == null, "The operator has already been initialized.");
+
+		initFileSystem();
+
+		if (this.refTruncate == null) {
+			this.refTruncate = reflectTruncate(fs);
+		}
+
+		OperatorStateStore stateStore = context.getOperatorStateStore();
+		restoredBucketStates = stateStore.getSerializableListState("bucket-states");
+
+		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+		if (context.isRestored()) {
+			LOG.info("Restoring state for the {} (taskIdx={}).", getClass().getSimpleName(), subtaskIndex);
+
+			for (State<T> recoveredState : restoredBucketStates.get()) {
+				handleRestoredBucketState(recoveredState);
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("{} idx {} restored {}", getClass().getSimpleName(), subtaskIndex, recoveredState);
+				}
+			}
+		} else {
+			LOG.info("No state to restore for the {} (taskIdx={}).", getClass().getSimpleName(), subtaskIndex);
+		}
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+
+		state = new State<>();
+
+		processingTimeService =
+				((StreamingRuntimeContext) getRuntimeContext()).getProcessingTimeService();
+
+		long currentProcessingTime = processingTimeService.getCurrentProcessingTime();
+
+		processingTimeService.registerTimer(currentProcessingTime + inactiveBucketCheckInterval, this);
+
+		this.clock = new Clock() {
+			@Override
+			public long currentTimeMillis() {
+				return processingTimeService.getCurrentProcessingTime();
+			}
+		};
+	}
+
+	/**
+	 * Create a file system with the user-defined {@code HDFS} configuration.
+	 * @throws IOException
+	 */
+	private void initFileSystem() throws IOException {
+		if (fs != null) {
+			return;
+		}
+		org.apache.hadoop.conf.Configuration hadoopConf = HadoopFileSystem.getHadoopConfiguration();
+		if (fsConfig != null) {
+			String disableCacheName = String.format("fs.%s.impl.disable.cache", new Path(basePath).toUri().getScheme());
+			hadoopConf.setBoolean(disableCacheName, true);
+			for (String key : fsConfig.keySet()) {
+				hadoopConf.set(key, fsConfig.getString(key, null));
+			}
+		}
+
+		fs = new Path(basePath).getFileSystem(hadoopConf);
+	}
+
+	@Override
+	public void close() throws Exception {
+		for (Map.Entry<String, BucketState<T>> entry : state.bucketStates.entrySet()) {
+			closeCurrentPartFile(entry.getValue());
+		}
+	}
+
+	@Override
+	public void invoke(T value) throws Exception {
+		Path bucketPath = bucketer.getBucketPath(clock, new Path(basePath), value);
+
+		long currentProcessingTime = processingTimeService.getCurrentProcessingTime();
+
+		BucketState<T> bucketState = state.getBucketState(bucketPath);
+		if (bucketState == null) {
+			bucketState = new BucketState<>(currentProcessingTime);
+			state.addBucketState(bucketPath, bucketState);
+		}
+
+		if (shouldRoll(bucketState)) {
+			openNewPartFile(bucketPath, bucketState);
+		}
+
+		bucketState.writer.write(value);
+		bucketState.lastWrittenToTime = currentProcessingTime;
+	}
+
+	/**
+	 * Returns {@code true} if the current {@code part-file} should be closed and a new should be created.
+	 * This happens if:
+	 * <ol>
+	 *     <li>no file is created yet for the task to write to, or</li>
+	 *     <li>the current file has reached the maximum bucket size.</li>
+	 * </ol>
+	 */
+	private boolean shouldRoll(BucketState<T> bucketState) throws IOException {
+		boolean shouldRoll = false;
+		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+		if (!bucketState.isWriterOpen) {
+			shouldRoll = true;
+			LOG.debug("BucketingSink {} starting new bucket.", subtaskIndex);
+		} else {
+			long writePosition = bucketState.writer.getPos();
+			if (writePosition > batchSize) {
+				shouldRoll = true;
+				LOG.debug(
+					"BucketingSink {} starting new bucket because file position {} is above batch size {}.",
+					subtaskIndex,
+					writePosition,
+					batchSize);
+			}
+		}
+		return shouldRoll;
+	}
+
+	@Override
+	public void onProcessingTime(long timestamp) throws Exception {
+		long currentProcessingTime = processingTimeService.getCurrentProcessingTime();
+
+		checkForInactiveBuckets(currentProcessingTime);
+
+		processingTimeService.registerTimer(currentProcessingTime + inactiveBucketCheckInterval, this);
+	}
+
+	/**
+	 * Checks for inactive buckets, and closes them. Buckets are considered inactive if they have not been
+	 * written to for a period greater than {@code inactiveBucketThreshold} ms. This enables in-progress
+	 * files to be moved to the pending state and be finalised on the next checkpoint.
+	 */
+	private void checkForInactiveBuckets(long currentProcessingTime) throws Exception {
+
+		synchronized (state.bucketStates) {
+			for (Map.Entry<String, BucketState<T>> entry : state.bucketStates.entrySet()) {
+				if (entry.getValue().lastWrittenToTime < currentProcessingTime - inactiveBucketThreshold) {
+					LOG.debug("BucketingSink {} closing bucket due to inactivity of over {} ms.",
+						getRuntimeContext().getIndexOfThisSubtask(), inactiveBucketThreshold);
+					closeCurrentPartFile(entry.getValue());
+				}
+			}
+		}
+	}
+
+	/**
+	 * Closes the current part file and opens a new one with a new bucket path, as returned by the
+	 * {@link Bucketer}. If the bucket is not new, then this will create a new file with the same path
+	 * as its predecessor, but with an increased rolling counter (see {@link BucketingSink}.
+	 */
+	private void openNewPartFile(Path bucketPath, BucketState<T> bucketState) throws Exception {
+		closeCurrentPartFile(bucketState);
+
+		if (!fs.exists(bucketPath)) {
+			try {
+				if (fs.mkdirs(bucketPath)) {
+					LOG.debug("Created new bucket directory: {}", bucketPath);
+				}
+			} catch (IOException e) {
+				throw new RuntimeException("Could not create new bucket path.", e);
+			}
+		}
+
+		// The following loop tries different partCounter values in ascending order until it reaches the minimum
+		// that is not yet used. This works since there is only one parallel subtask that tries names with this
+		// subtask id. Otherwise we would run into concurrency issues here. This is aligned with the way we now
+		// clean the base directory in case of rescaling.
+
+		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+		Path partPath = new Path(bucketPath, partPrefix + "-" + subtaskIndex + "-" + bucketState.partCounter);
+		while (fs.exists(partPath) ||
+				fs.exists(getPendingPathFor(partPath)) ||
+				fs.exists(getInProgressPathFor(partPath))) {
+			bucketState.partCounter++;
+			partPath = new Path(bucketPath, partPrefix + "-" + subtaskIndex + "-" + bucketState.partCounter);
+		}
+
+		// increase, so we don't have to check for this name next time
+		bucketState.partCounter++;
+
+		LOG.debug("Next part path is {}", partPath.toString());
+		bucketState.currentFile = partPath.toString();
+
+		Path inProgressPath = getInProgressPathFor(partPath);
+		if (bucketState.writer == null) {
+			bucketState.writer = writerTemplate.duplicate();
+		}
+
+		bucketState.writer.open(fs, inProgressPath);
+		bucketState.isWriterOpen = true;
+	}
+
+	/**
+	 * Closes the current part file and moves it from the in-progress state to the pending state.
+	 */
+	private void closeCurrentPartFile(BucketState<T> bucketState) throws Exception {
+		if (bucketState.isWriterOpen) {
+			bucketState.writer.close();
+			bucketState.isWriterOpen = false;
+		}
+
+		if (bucketState.currentFile != null) {
+			Path currentPartPath = new Path(bucketState.currentFile);
+			Path inProgressPath = getInProgressPathFor(currentPartPath);
+			Path pendingPath = getPendingPathFor(currentPartPath);
+
+			fs.rename(inProgressPath, pendingPath);
+			LOG.debug("Moving in-progress bucket {} to pending file {}",
+				inProgressPath,
+				pendingPath);
+			bucketState.pendingFiles.add(currentPartPath.toString());
+			bucketState.currentFile = null;
+		}
+	}
+
+	/**
+	 * Gets the truncate() call using reflection.
+	 * <p>
+	 * <b>NOTE:</b> This code comes from Flume.
+	 */
+	private Method reflectTruncate(FileSystem fs) {
+		Method m = null;
+		if(fs != null) {
+			Class<?> fsClass = fs.getClass();
+			try {
+				m = fsClass.getMethod("truncate", Path.class, long.class);
+			} catch (NoSuchMethodException ex) {
+				LOG.debug("Truncate not found. Will write a file with suffix '{}' " +
+					" and prefix '{}' to specify how many bytes in a bucket are valid.", validLengthSuffix, validLengthPrefix);
+				return null;
+			}
+
+			// verify that truncate actually works
+			FSDataOutputStream outputStream;
+			Path testPath = new Path(UUID.randomUUID().toString());
+			try {
+				outputStream = fs.create(testPath);
+				outputStream.writeUTF("hello");
+				outputStream.close();
+			} catch (IOException e) {
+				LOG.error("Could not create file for checking if truncate works.", e);
+				throw new RuntimeException("Could not create file for checking if truncate works.", e);
+			}
+
+			try {
+				m.invoke(fs, testPath, 2);
+			} catch (IllegalAccessException | InvocationTargetException e) {
+				LOG.debug("Truncate is not supported.", e);
+				m = null;
+			}
+
+			try {
+				fs.delete(testPath, false);
+			} catch (IOException e) {
+				LOG.error("Could not delete truncate test file.", e);
+				throw new RuntimeException("Could not delete truncate test file.", e);
+			}
+		}
+		return m;
+	}
+
+	private Path getPendingPathFor(Path path) {
+		return new Path(path.getParent(), pendingPrefix + path.getName()).suffix(pendingSuffix);
+	}
+
+	private Path getInProgressPathFor(Path path) {
+		return new Path(path.getParent(), inProgressPrefix + path.getName()).suffix(inProgressSuffix);
+	}
+
+	private Path getValidLengthPathFor(Path path) {
+		return new Path(path.getParent(), validLengthPrefix + path.getName()).suffix(validLengthSuffix);
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {
+		synchronized (state.bucketStates) {
+
+			Iterator<Map.Entry<String, BucketState<T>>> bucketStatesIt = state.bucketStates.entrySet().iterator();
+			while (bucketStatesIt.hasNext()) {
+				BucketState<T> bucketState = bucketStatesIt.next().getValue();
+				synchronized (bucketState.pendingFilesPerCheckpoint) {
+
+					Iterator<Map.Entry<Long, List<String>>> pendingCheckpointsIt =
+						bucketState.pendingFilesPerCheckpoint.entrySet().iterator();
+
+					while (pendingCheckpointsIt.hasNext()) {
+
+						Map.Entry<Long, List<String>> entry = pendingCheckpointsIt.next();
+						Long pastCheckpointId = entry.getKey();
+						List<String> pendingPaths = entry.getValue();
+
+						if (pastCheckpointId <= checkpointId) {
+							LOG.debug("Moving pending files to final location for checkpoint {}", pastCheckpointId);
+
+							for (String filename : pendingPaths) {
+								Path finalPath = new Path(filename);
+								Path pendingPath = getPendingPathFor(finalPath);
+
+								fs.rename(pendingPath, finalPath);
+								LOG.debug(
+									"Moving pending file {} to final location having completed checkpoint {}.",
+									pendingPath,
+									pastCheckpointId);
+							}
+							pendingCheckpointsIt.remove();
+						}
+					}
+
+					if (!bucketState.isWriterOpen &&
+						bucketState.pendingFiles.isEmpty() &&
+						bucketState.pendingFilesPerCheckpoint.isEmpty()) {
+
+						// We've dealt with all the pending files and the writer for this bucket is not currently open.
+						// Therefore this bucket is currently inactive and we can remove it from our state.
+						bucketStatesIt.remove();
+					}
+				}
+			}
+		}
+	}
+
+	@Override
+	public void snapshotState(FunctionSnapshotContext context) throws Exception {
+		Preconditions.checkNotNull(restoredBucketStates, "The operator has not been properly initialized.");
+
+		restoredBucketStates.clear();
+
+		synchronized (state.bucketStates) {
+			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
+
+			for (Map.Entry<String, BucketState<T>> bucketStateEntry : state.bucketStates.entrySet()) {
+				BucketState<T> bucketState = bucketStateEntry.getValue();
+
+				if (bucketState.isWriterOpen) {
+					bucketState.currentFileValidLength = bucketState.writer.flush();
+				}
+
+				synchronized (bucketState.pendingFilesPerCheckpoint) {
+					bucketState.pendingFilesPerCheckpoint.put(context.getCheckpointId(), bucketState.pendingFiles);
+				}
+				bucketState.pendingFiles = new ArrayList<>();
+			}
+			restoredBucketStates.add(state);
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("{} idx {} checkpointed {}.", getClass().getSimpleName(), subtaskIdx, state);
+			}
+		}
+	}
+
+	private void handleRestoredBucketState(State<T> restoredState) {
+		Preconditions.checkNotNull(restoredState);
+
+		for (BucketState<T> bucketState : restoredState.bucketStates.values()) {
+
+			// we can clean all the pending files since they were renamed to
+			// final files after this checkpoint was successful
+			// (we re-start from the last **successful** checkpoint)
+			bucketState.pendingFiles.clear();
+
+			if (bucketState.currentFile != null) {
+
+				// We were writing to a file when the last checkpoint occurred. This file can either
+				// be still in-progress or became a pending file at some point after the checkpoint.
+				// Either way, we have to truncate it back to a valid state (or write a .valid-length
+				// file that specifies up to which length it is valid) and rename it to the final name
+				// before starting a new bucket file.
+
+				Path partPath = new Path(bucketState.currentFile);
+				try {
+					Path partPendingPath = getPendingPathFor(partPath);
+					Path partInProgressPath = getInProgressPathFor(partPath);
+
+					if (fs.exists(partPendingPath)) {
+						LOG.debug("In-progress file {} has been moved to pending after checkpoint, moving to final location.", partPath);
+						// has been moved to pending in the mean time, rename to final location
+						fs.rename(partPendingPath, partPath);
+					} else if (fs.exists(partInProgressPath)) {
+						LOG.debug("In-progress file {} is still in-progress, moving to final location.", partPath);
+						// it was still in progress, rename to final path
+						fs.rename(partInProgressPath, partPath);
+					} else if (fs.exists(partPath)) {
+						LOG.debug("In-Progress file {} was already moved to final location {}.", bucketState.currentFile, partPath);
+					} else {
+						LOG.debug("In-Progress file {} was neither moved to pending nor is still in progress. Possibly, " +
+							"it was moved to final location by a previous snapshot restore", bucketState.currentFile);
+					}
+
+					// We use reflection to get the .truncate() method, this
+					// is only available starting with Hadoop 2.7
+					if (this.refTruncate == null) {
+						this.refTruncate = reflectTruncate(fs);
+					}
+
+					// truncate it or write a ".valid-length" file to specify up to which point it is valid
+					if (refTruncate != null) {
+						LOG.debug("Truncating {} to valid length {}", partPath, bucketState.currentFileValidLength);
+						// some-one else might still hold the lease from a previous try, we are
+						// recovering, after all ...
+						if (fs instanceof DistributedFileSystem) {
+							DistributedFileSystem dfs = (DistributedFileSystem) fs;
+							LOG.debug("Trying to recover file lease {}", partPath);
+							dfs.recoverLease(partPath);
+							boolean isclosed = dfs.isFileClosed(partPath);
+							StopWatch sw = new StopWatch();
+							sw.start();
+							while (!isclosed) {
+								if (sw.getTime() > asyncTimeout) {
+									break;
+								}
+								try {
+									Thread.sleep(500);
+								} catch (InterruptedException e1) {
+									// ignore it
+								}
+								isclosed = dfs.isFileClosed(partPath);
+							}
+						}
+						Boolean truncated = (Boolean) refTruncate.invoke(fs, partPath, bucketState.currentFileValidLength);
+						if (!truncated) {
+							LOG.debug("Truncate did not immediately complete for {}, waiting...", partPath);
+
+							// we must wait for the asynchronous truncate operation to complete
+							StopWatch sw = new StopWatch();
+							sw.start();
+							long newLen = fs.getFileStatus(partPath).getLen();
+							while (newLen != bucketState.currentFileValidLength) {
+								if (sw.getTime() > asyncTimeout) {
+									break;
+								}
+								try {
+									Thread.sleep(500);
+								} catch (InterruptedException e1) {
+									// ignore it
+								}
+								newLen = fs.getFileStatus(partPath).getLen();
+							}
+							if (newLen != bucketState.currentFileValidLength) {
+								throw new RuntimeException("Truncate did not truncate to right length. Should be " + bucketState.currentFileValidLength + " is " + newLen + ".");
+							}
+						}
+					} else {
+						LOG.debug("Writing valid-length file for {} to specify valid length {}", partPath, bucketState.currentFileValidLength);
+						Path validLengthFilePath = getValidLengthPathFor(partPath);
+						if (!fs.exists(validLengthFilePath)) {
+							FSDataOutputStream lengthFileOut = fs.create(validLengthFilePath);
+							lengthFileOut.writeUTF(Long.toString(bucketState.currentFileValidLength));
+							lengthFileOut.close();
+						}
+					}
+
+					// Now that we've restored the bucket to a valid state, reset the current file info
+					bucketState.currentFile = null;
+					bucketState.currentFileValidLength = -1;
+					bucketState.isWriterOpen = false;
+				} catch (IOException e) {
+					LOG.error("Error while restoring BucketingSink state.", e);
+					throw new RuntimeException("Error while restoring BucketingSink state.", e);
+				} catch (InvocationTargetException | IllegalAccessException e) {
+					LOG.error("Could not invoke truncate.", e);
+					throw new RuntimeException("Could not invoke truncate.", e);
+				}
+			}
+
+			// Move files that are confirmed by a checkpoint but did not get moved to final location
+			// because the checkpoint notification did not happen before a failure
+
+			LOG.debug("Moving pending files to final location on restore.");
+
+			Set<Long> pastCheckpointIds = bucketState.pendingFilesPerCheckpoint.keySet();
+			for (Long pastCheckpointId : pastCheckpointIds) {
+				// All the pending files are buckets that have been completed but are waiting to be renamed
+				// to their final name
+				for (String filename : bucketState.pendingFilesPerCheckpoint.get(pastCheckpointId)) {
+					Path finalPath = new Path(filename);
+					Path pendingPath = getPendingPathFor(finalPath);
+
+					try {
+						if (fs.exists(pendingPath)) {
+							LOG.debug("Restoring BucketingSink State: Moving pending file {} to final location after complete checkpoint {}.", pendingPath, pastCheckpointId);
+							fs.rename(pendingPath, finalPath);
+						}
+					} catch (IOException e) {
+						LOG.error("Restoring BucketingSink State: Error while renaming pending file {} to final path {}: {}", pendingPath, finalPath, e);
+						throw new RuntimeException("Error while renaming pending file " + pendingPath + " to final path " + finalPath, e);
+					}
+				}
+			}
+
+			synchronized (bucketState.pendingFilesPerCheckpoint) {
+				bucketState.pendingFilesPerCheckpoint.clear();
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Setters for User configuration values
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Sets the maximum bucket size in bytes.
+	 *
+	 * <p>
+	 * When a bucket part file becomes larger than this size a new bucket part file is started and
+	 * the old one is closed. The name of the bucket files depends on the {@link Bucketer}.
+	 *
+	 * @param batchSize The bucket part file size in bytes.
+	 */
+	public BucketingSink<T> setBatchSize(long batchSize) {
+		this.batchSize = batchSize;
+		return this;
+	}
+
+	/**
+	 * Sets the default time between checks for inactive buckets.
+	 *
+	 * @param interval The timeout, in milliseconds.
+	 */
+	public BucketingSink<T> setInactiveBucketCheckInterval(long interval) {
+		this.inactiveBucketCheckInterval = interval;
+		return this;
+	}
+
+	/**
+	 * Sets the default threshold for marking a bucket as inactive and closing its part files.
+	 * Buckets which haven't been written to for at least this period of time become inactive.
+	 *
+	 * @param threshold The timeout, in milliseconds.
+	 */
+	public BucketingSink<T> setInactiveBucketThreshold(long threshold) {
+		this.inactiveBucketThreshold = threshold;
+		return this;
+	}
+
+	/**
+	 * Sets the {@link Bucketer} to use for determining the bucket files to write to.
+	 *
+	 * @param bucketer The bucketer to use.
+	 */
+	public BucketingSink<T> setBucketer(Bucketer<T> bucketer) {
+		this.bucketer = bucketer;
+		return this;
+	}
+
+	/**
+	 * Sets the {@link Writer} to be used for writing the incoming elements to bucket files.
+	 *
+	 * @param writer The {@code Writer} to use.
+	 */
+	public BucketingSink<T> setWriter(Writer<T> writer) {
+		this.writerTemplate = writer;
+		return this;
+	}
+
+	/**
+	 * Sets the suffix of in-progress part files. The default is {@code "in-progress"}.
+	 */
+	public BucketingSink<T> setInProgressSuffix(String inProgressSuffix) {
+		this.inProgressSuffix = inProgressSuffix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of in-progress part files. The default is {@code "_"}.
+	 */
+	public BucketingSink<T> setInProgressPrefix(String inProgressPrefix) {
+		this.inProgressPrefix = inProgressPrefix;
+		return this;
+	}
+
+	/**
+	 * Sets the suffix of pending part files. The default is {@code ".pending"}.
+	 */
+	public BucketingSink<T> setPendingSuffix(String pendingSuffix) {
+		this.pendingSuffix = pendingSuffix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of pending part files. The default is {@code "_"}.
+	 */
+	public BucketingSink<T> setPendingPrefix(String pendingPrefix) {
+		this.pendingPrefix = pendingPrefix;
+		return this;
+	}
+
+	/**
+	 * Sets the suffix of valid-length files. The default is {@code ".valid-length"}.
+	 */
+	public BucketingSink<T> setValidLengthSuffix(String validLengthSuffix) {
+		this.validLengthSuffix = validLengthSuffix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of valid-length files. The default is {@code "_"}.
+	 */
+	public BucketingSink<T> setValidLengthPrefix(String validLengthPrefix) {
+		this.validLengthPrefix = validLengthPrefix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of part files.  The default is {@code "part"}.
+	 */
+	public BucketingSink<T> setPartPrefix(String partPrefix) {
+		this.partPrefix = partPrefix;
+		return this;
+	}
+
+	/**
+	 * Disable cleanup of leftover in-progress/pending files when the sink is opened.
+	 *
+	 * <p>
+	 * This should only be disabled if using the sink without checkpoints, to not remove
+	 * the files already in the directory.
+	 *
+	 * @deprecated This option is deprecated and remains only for backwards compatibility.
+	 * We do not clean up lingering files anymore.
+	 */
+	@Deprecated
+	public BucketingSink<T> disableCleanupOnOpen() {
+		return this;
+	}
+
+	/**
+	 * Sets the default timeout for asynchronous operations such as recoverLease and truncate.
+	 *
+	 * @param timeout The timeout, in milliseconds.
+	 */
+	public BucketingSink<T> setAsyncTimeout(long timeout) {
+		this.asyncTimeout = timeout;
+		return this;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Internal Classes
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * This is used during snapshot/restore to keep track of in-progress buckets.
+	 * For each bucket, we maintain a state.
+	 */
+	static final class State<T> implements Serializable {
+		private static final long serialVersionUID = 1L;
+
+		/**
+		 * For every bucket directory (key), we maintain a bucket state (value).
+		 */
+		final Map<String, BucketState<T>> bucketStates = new HashMap<>();
+
+		void addBucketState(Path bucketPath, BucketState<T> state) {
+			synchronized (bucketStates) {
+				bucketStates.put(bucketPath.toString(), state);
+			}
+		}
+
+		BucketState<T> getBucketState(Path bucketPath) {
+			synchronized (bucketStates) {
+				return bucketStates.get(bucketPath.toString());
+			}
+		}
+
+		@Override
+		public String toString() {
+			return bucketStates.toString();
+		}
+	}
+
+	/**
+	 * This is used for keeping track of the current in-progress buckets and files that we mark
+	 * for moving from pending to final location after we get a checkpoint-complete notification.
+	 */
+	static final class BucketState<T> implements Serializable {
+		private static final long serialVersionUID = 1L;
+
+		/**
+		 * The file that was in-progress when the last checkpoint occurred.
+		 */
+		String currentFile;
+
+		/**
+		 * The valid length of the in-progress file at the time of the last checkpoint.
+		 */
+		long currentFileValidLength = -1;
+
+		/**
+		 * The time this bucket was last written to.
+		 */
+		long lastWrittenToTime;
+
+		/**
+		 * Pending files that accumulated since the last checkpoint.
+		 */
+		List<String> pendingFiles = new ArrayList<>();
+
+		/**
+		 * When doing a checkpoint we move the pending files since the last checkpoint to this map
+		 * with the id of the checkpoint. When we get the checkpoint-complete notification we move
+		 * pending files of completed checkpoints to their final location.
+		 */
+		final Map<Long, List<String>> pendingFilesPerCheckpoint = new HashMap<>();
+
+		/**
+		 * For counting the part files inside a bucket directory. Part files follow the pattern
+		 * {@code "{part-prefix}-{subtask}-{count}"}. When creating new part files we increase the counter.
+		 */
+		private transient int partCounter;
+
+		/**
+		 * Tracks if the writer is currently opened or closed.
+		 */
+		private transient boolean isWriterOpen;
+
+		/**
+		 * The actual writer that we user for writing the part files.
+		 */
+		private transient Writer<T> writer;
+
+		@Override
+		public String toString() {
+			return
+				"In-progress=" + currentFile +
+					" validLength=" + currentFileValidLength +
+					" pendingForNextCheckpoint=" + pendingFiles +
+					" pendingForPrevCheckpoints=" + pendingFilesPerCheckpoint +
+					" lastModified@" + lastWrittenToTime;
+		}
+
+		BucketState(long lastWrittenToTime) {
+			this.lastWrittenToTime = lastWrittenToTime;
+		}
+	}
+}

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

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

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


[06/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java
deleted file mode 100644
index 51bc8d1..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.runtime.util.DataInputDeserializer;
-import org.apache.flink.runtime.util.DataOutputSerializer;
-
-import java.io.IOException;
-
-/**
- * A serialization and deserialization schema for Key Value Pairs that uses Flink's serialization stack to
- * transform typed from and to byte arrays.
- * 
- * @param <K> The key type to be serialized.
- * @param <V> The value type to be serialized.
- */
-public class TypeInformationKeyValueSerializationSchema<K, V> implements KeyedDeserializationSchema<Tuple2<K, V>>, KeyedSerializationSchema<Tuple2<K,V>> {
-
-	private static final long serialVersionUID = -5359448468131559102L;
-
-	/** The serializer for the key */
-	private final TypeSerializer<K> keySerializer;
-
-	/** The serializer for the value */
-	private final TypeSerializer<V> valueSerializer;
-
-	/** reusable input deserialization buffer */
-	private final DataInputDeserializer inputDeserializer;
-	
-	/** reusable output serialization buffer for the key */
-	private transient DataOutputSerializer keyOutputSerializer;
-
-	/** reusable output serialization buffer for the value */
-	private transient DataOutputSerializer valueOutputSerializer;
-	
-	
-	/** The type information, to be returned by {@link #getProducedType()}. It is
-	 * transient, because it is not serializable. Note that this means that the type information
-	 * is not available at runtime, but only prior to the first serialization / deserialization */
-	private final transient TypeInformation<Tuple2<K, V>> typeInfo;
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new de-/serialization schema for the given types.
-	 *
-	 * @param keyTypeInfo The type information for the key type de-/serialized by this schema.
-	 * @param valueTypeInfo The type information for the value type de-/serialized by this schema.
-	 * @param ec The execution config, which is used to parametrize the type serializers.
-	 */
-	public TypeInformationKeyValueSerializationSchema(TypeInformation<K> keyTypeInfo, TypeInformation<V> valueTypeInfo, ExecutionConfig ec) {
-		this.typeInfo = new TupleTypeInfo<>(keyTypeInfo, valueTypeInfo);
-		this.keySerializer = keyTypeInfo.createSerializer(ec);
-		this.valueSerializer = valueTypeInfo.createSerializer(ec);
-		this.inputDeserializer = new DataInputDeserializer();
-	}
-
-	/**
-	 * Creates a new de-/serialization schema for the given types. This constructor accepts the types
-	 * as classes and internally constructs the type information from the classes.
-	 * 
-	 * <p>If the types are parametrized and cannot be fully defined via classes, use the constructor
-	 * that accepts {@link TypeInformation} instead.
-	 * 
-	 * @param keyClass The class of the key de-/serialized by this schema.
-	 * @param valueClass The class of the value de-/serialized by this schema.
-	 * @param config The execution config, which is used to parametrize the type serializers.
-	 */
-	public TypeInformationKeyValueSerializationSchema(Class<K> keyClass, Class<V> valueClass, ExecutionConfig config) {
-		this(TypeExtractor.createTypeInfo(keyClass), TypeExtractor.createTypeInfo(valueClass), config);
-	}
-
-	// ------------------------------------------------------------------------
-
-
-	@Override
-	public Tuple2<K, V> deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
-		K key = null;
-		V value = null;
-		
-		if (messageKey != null) {
-			inputDeserializer.setBuffer(messageKey, 0, messageKey.length);
-			key = keySerializer.deserialize(inputDeserializer);
-		}
-		if (message != null) {
-			inputDeserializer.setBuffer(message, 0, message.length);
-			value = valueSerializer.deserialize(inputDeserializer);
-		}
-		return new Tuple2<>(key, value);
-	}
-
-	/**
-	 * This schema never considers an element to signal end-of-stream, so this method returns always false.
-	 * @param nextElement The element to test for the end-of-stream signal.
-	 * @return Returns false.
-	 */
-	@Override
-	public boolean isEndOfStream(Tuple2<K,V> nextElement) {
-		return false;
-	}
-
-
-	@Override
-	public byte[] serializeKey(Tuple2<K, V> element) {
-		if (element.f0 == null) {
-			return null;
-		} else {
-			// key is not null. serialize it:
-			if (keyOutputSerializer == null) {
-				keyOutputSerializer = new DataOutputSerializer(16);
-			}
-			try {
-				keySerializer.serialize(element.f0, keyOutputSerializer);
-			}
-			catch (IOException e) {
-				throw new RuntimeException("Unable to serialize record", e);
-			}
-			// check if key byte array size changed
-			byte[] res = keyOutputSerializer.getByteArray();
-			if (res.length != keyOutputSerializer.length()) {
-				byte[] n = new byte[keyOutputSerializer.length()];
-				System.arraycopy(res, 0, n, 0, keyOutputSerializer.length());
-				res = n;
-			}
-			keyOutputSerializer.clear();
-			return res;
-		}
-	}
-
-	@Override
-	public byte[] serializeValue(Tuple2<K, V> element) {
-		// if the value is null, its serialized value is null as well.
-		if (element.f1 == null) {
-			return null;
-		}
-
-		if (valueOutputSerializer == null) {
-			valueOutputSerializer = new DataOutputSerializer(16);
-		}
-
-		try {
-			valueSerializer.serialize(element.f1, valueOutputSerializer);
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Unable to serialize record", e);
-		}
-
-		byte[] res = valueOutputSerializer.getByteArray();
-		if (res.length != valueOutputSerializer.length()) {
-			byte[] n = new byte[valueOutputSerializer.length()];
-			System.arraycopy(res, 0, n, 0, valueOutputSerializer.length());
-			res = n;
-		}
-		valueOutputSerializer.clear();
-		return res;
-	}
-
-	@Override
-	public String getTargetTopic(Tuple2<K, V> element) {
-		return null; // we are never overriding the topic
-	}
-
-
-	@Override
-	public TypeInformation<Tuple2<K,V>> getProducedType() {
-		if (typeInfo != null) {
-			return typeInfo;
-		}
-		else {
-			throw new IllegalStateException(
-					"The type information is not available after this class has been serialized and distributed.");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index b96ba30..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
+++ /dev/null
@@ -1,416 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.commons.collections.map.LinkedMap;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.state.StateInitializationContext;
-import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
-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;
-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.Assert;
-import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.Serializable;
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class FlinkKafkaConsumerBaseTest {
-
-	/**
-	 * Tests that not both types of timestamp extractors / watermark generators can be used.
-	 */
-	@Test
-	public void testEitherWatermarkExtractor() {
-		try {
-			new DummyFlinkKafkaConsumer<>().assignTimestampsAndWatermarks((AssignerWithPeriodicWatermarks<Object>) null);
-			fail();
-		} catch (NullPointerException ignored) {}
-
-		try {
-			new DummyFlinkKafkaConsumer<>().assignTimestampsAndWatermarks((AssignerWithPunctuatedWatermarks<Object>) 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.assignTimestampsAndWatermarks(periodicAssigner);
-		try {
-			c1.assignTimestampsAndWatermarks(punctuatedAssigner);
-			fail();
-		} catch (IllegalStateException ignored) {}
-
-		DummyFlinkKafkaConsumer<String> c2 = new DummyFlinkKafkaConsumer<>();
-		c2.assignTimestampsAndWatermarks(punctuatedAssigner);
-		try {
-			c2.assignTimestampsAndWatermarks(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);
-		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
-		TestingListState<Tuple2<KafkaTopicPartition, Long>> listState = new TestingListState<>();
-		when(operatorStateStore.getOperatorState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState);
-
-		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(1, 1));
-
-		assertFalse(listState.get().iterator().hasNext());
-		consumer.notifyCheckpointComplete(66L);
-	}
-
-	/**
-	 * Tests that no checkpoints happen when the fetcher is not running.
-	 */
-	@Test
-	public void checkRestoredCheckpointWhenFetcherNotReady() throws Exception {
-		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
-
-		TestingListState<Serializable> listState = new TestingListState<>();
-		listState.add(Tuple2.of(new KafkaTopicPartition("abc", 13), 16768L));
-		listState.add(Tuple2.of(new KafkaTopicPartition("def", 7), 987654321L));
-
-		FlinkKafkaConsumerBase<String> consumer = getConsumer(null, new LinkedMap(), true);
-
-		when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(listState);
-
-		StateInitializationContext initializationContext = mock(StateInitializationContext.class);
-
-		when(initializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
-		when(initializationContext.isRestored()).thenReturn(true);
-
-		consumer.initializeState(initializationContext);
-
-		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(17, 17));
-
-		// ensure that the list was cleared and refilled. while this is an implementation detail, we use it here
-		// to figure out that snapshotState() actually did something.
-		Assert.assertTrue(listState.isClearCalled());
-
-		Set<Serializable> expected = new HashSet<>();
-
-		for (Serializable serializable : listState.get()) {
-			expected.add(serializable);
-		}
-
-		int counter = 0;
-
-		for (Serializable serializable : listState.get()) {
-			assertTrue(expected.contains(serializable));
-			counter++;
-		}
-
-		assertEquals(expected.size(), counter);
-	}
-
-	/**
-	 * 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);
-
-		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
-		TestingListState<Serializable> listState = new TestingListState<>();
-		when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(listState);
-
-		StateInitializationContext initializationContext = mock(StateInitializationContext.class);
-
-		when(initializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
-		when(initializationContext.isRestored()).thenReturn(false);
-
-		consumer.initializeState(initializationContext);
-
-		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(17, 17));
-
-		assertFalse(listState.get().iterator().hasNext());
-	}
-
-	/**
-	 * Tests that on snapshots, states and offsets to commit to Kafka are correct
-	 */
-	@Test
-	public void checkUseFetcherWhenNoCheckpoint() throws Exception {
-
-		FlinkKafkaConsumerBase<String> consumer = getConsumer(null, new LinkedMap(), true);
-		List<KafkaTopicPartition> partitionList = new ArrayList<>(1);
-		partitionList.add(new KafkaTopicPartition("test", 0));
-		consumer.setSubscribedPartitions(partitionList);
-
-		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
-		TestingListState<Serializable> listState = new TestingListState<>();
-		when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(listState);
-
-		StateInitializationContext initializationContext = mock(StateInitializationContext.class);
-
-		when(initializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
-
-		// make the context signal that there is no restored state, then validate that
-		when(initializationContext.isRestored()).thenReturn(false);
-		consumer.initializeState(initializationContext);
-		consumer.run(mock(SourceFunction.SourceContext.class));
-	}
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testSnapshotState() throws Exception {
-
-		// --------------------------------------------------------------------
-		//   prepare fake states
-		// --------------------------------------------------------------------
-
-		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<>();
-		state3.put(new KafkaTopicPartition("abc", 13), 16780L);
-		state3.put(new KafkaTopicPartition("def", 7), 987654377L);
-
-		// --------------------------------------------------------------------
-		
-		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
-		when(fetcher.snapshotCurrentState()).thenReturn(state1, state2, state3);
-			
-		final LinkedMap pendingOffsetsToCommit = new LinkedMap();
-	
-		FlinkKafkaConsumerBase<String> consumer = getConsumer(fetcher, pendingOffsetsToCommit, true);
-		assertEquals(0, pendingOffsetsToCommit.size());
-
-		OperatorStateStore backend = mock(OperatorStateStore.class);
-
-		TestingListState<Serializable> listState = new TestingListState<>();
-
-		when(backend.getSerializableListState(Matchers.any(String.class))).thenReturn(listState);
-
-		StateInitializationContext initializationContext = mock(StateInitializationContext.class);
-
-		when(initializationContext.getOperatorStateStore()).thenReturn(backend);
-		when(initializationContext.isRestored()).thenReturn(false, true, true, true);
-
-		consumer.initializeState(initializationContext);
-
-		// checkpoint 1
-		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(138, 138));
-
-		HashMap<KafkaTopicPartition, Long> snapshot1 = new HashMap<>();
-
-		for (Serializable serializable : listState.get()) {
-			Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 = (Tuple2<KafkaTopicPartition, Long>) serializable;
-			snapshot1.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
-		}
-
-		assertEquals(state1, snapshot1);
-		assertEquals(1, pendingOffsetsToCommit.size());
-		assertEquals(state1, pendingOffsetsToCommit.get(138L));
-
-		// checkpoint 2
-		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(140, 140));
-
-		HashMap<KafkaTopicPartition, Long> snapshot2 = new HashMap<>();
-
-		for (Serializable serializable : listState.get()) {
-			Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 = (Tuple2<KafkaTopicPartition, Long>) serializable;
-			snapshot2.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
-		}
-
-		assertEquals(state2, snapshot2);
-		assertEquals(2, pendingOffsetsToCommit.size());
-		assertEquals(state2, pendingOffsetsToCommit.get(140L));
-		
-		// ack checkpoint 1
-		consumer.notifyCheckpointComplete(138L);
-		assertEquals(1, pendingOffsetsToCommit.size());
-		assertTrue(pendingOffsetsToCommit.containsKey(140L));
-
-		// checkpoint 3
-		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(141, 141));
-
-		HashMap<KafkaTopicPartition, Long> snapshot3 = new HashMap<>();
-
-		for (Serializable serializable : listState.get()) {
-			Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 = (Tuple2<KafkaTopicPartition, Long>) serializable;
-			snapshot3.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
-		}
-
-		assertEquals(state3, snapshot3);
-		assertEquals(2, pendingOffsetsToCommit.size());
-		assertEquals(state3, pendingOffsetsToCommit.get(141L));
-		
-		// ack checkpoint 3, subsumes number 2
-		consumer.notifyCheckpointComplete(141L);
-		assertEquals(0, pendingOffsetsToCommit.size());
-
-
-		consumer.notifyCheckpointComplete(666); // invalid checkpoint
-		assertEquals(0, pendingOffsetsToCommit.size());
-
-		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
-		listState = new TestingListState<>();
-		when(operatorStateStore.getOperatorState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState);
-
-		// create 500 snapshots
-		for (int i = 100; i < 600; i++) {
-			consumer.snapshotState(new StateSnapshotContextSynchronousImpl(i, i));
-			listState.clear();
-		}
-		assertEquals(FlinkKafkaConsumerBase.MAX_NUM_PENDING_CHECKPOINTS, pendingOffsetsToCommit.size());
-
-		// commit only the second last
-		consumer.notifyCheckpointComplete(598);
-		assertEquals(1, pendingOffsetsToCommit.size());
-
-		// access invalid checkpoint
-		consumer.notifyCheckpointComplete(590);
-
-		// and the last
-		consumer.notifyCheckpointComplete(599);
-		assertEquals(0, pendingOffsetsToCommit.size());
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static <T> FlinkKafkaConsumerBase<T> getConsumer(
-			AbstractFetcher<T, ?> fetcher, LinkedMap pendingOffsetsToCommit, 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("pendingOffsetsToCommit");
-		mapField.setAccessible(true);
-		mapField.set(consumer, pendingOffsetsToCommit);
-
-		Field runningField = FlinkKafkaConsumerBase.class.getDeclaredField("running");
-		runningField.setAccessible(true);
-		runningField.set(consumer, running);
-
-		return consumer;
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
-		private static final long serialVersionUID = 1L;
-
-		@SuppressWarnings("unchecked")
-		public DummyFlinkKafkaConsumer() {
-			super(Arrays.asList("dummy-topic"), (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 {
-			AbstractFetcher<T, ?> fetcher = mock(AbstractFetcher.class);
-			doAnswer(new Answer() {
-				@Override
-				public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
-					Assert.fail("Trying to restore offsets even though there was no restore state.");
-					return null;
-				}
-			}).when(fetcher).restoreOffsets(any(HashMap.class));
-			return fetcher;
-		}
-
-		@Override
-		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
-			return Collections.emptyList();
-		}
-
-		@Override
-		public RuntimeContext getRuntimeContext() {
-			return mock(StreamingRuntimeContext.class);
-		}
-	}
-
-	private static final class TestingListState<T> implements ListState<T> {
-
-		private final List<T> list = new ArrayList<>();
-		private boolean clearCalled = false;
-
-		@Override
-		public void clear() {
-			list.clear();
-			clearCalled = true;
-		}
-
-		@Override
-		public Iterable<T> get() throws Exception {
-			return list;
-		}
-
-		@Override
-		public void add(T value) throws Exception {
-			list.add(value);
-		}
-
-		public List<T> getList() {
-			return list;
-		}
-
-		public boolean isClearCalled() {
-			return clearCalled;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
deleted file mode 100644
index 2e06160..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
+++ /dev/null
@@ -1,288 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.junit.Assert;
-import org.junit.Test;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-public class FlinkKafkaProducerBaseTest {
-
-	/**
-	 * Tests that the constructor eagerly checks bootstrap servers are set in config
-	 */
-	@Test(expected = IllegalArgumentException.class)
-	public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception {
-		// no bootstrap servers set in props
-		Properties props = new Properties();
-		// should throw IllegalArgumentException
-		new DummyFlinkKafkaProducer<>(props, null);
-	}
-
-	/**
-	 * Tests that constructor defaults to key value serializers in config to byte array deserializers if not set
-	 */
-	@Test
-	public void testKeyValueDeserializersSetIfMissing() throws Exception {
-		Properties props = new Properties();
-		props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:12345");
-		// should set missing key value deserializers
-		new DummyFlinkKafkaProducer<>(props, null);
-
-		assertTrue(props.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG));
-		assertTrue(props.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG));
-		assertTrue(props.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).equals(ByteArraySerializer.class.getCanonicalName()));
-		assertTrue(props.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).equals(ByteArraySerializer.class.getCanonicalName()));
-	}
-
-	/**
-	 * Tests that partitions list is determinate and correctly provided to custom partitioner
-	 */
-	@Test
-	public void testPartitionerOpenedWithDeterminatePartitionList() throws Exception {
-		KafkaPartitioner mockPartitioner = mock(KafkaPartitioner.class);
-		RuntimeContext mockRuntimeContext = mock(RuntimeContext.class);
-		when(mockRuntimeContext.getIndexOfThisSubtask()).thenReturn(0);
-		when(mockRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(1);
-
-		DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer(
-			FakeStandardProducerConfig.get(), mockPartitioner);
-		producer.setRuntimeContext(mockRuntimeContext);
-
-		producer.open(new Configuration());
-
-		// the internal mock KafkaProducer will return an out-of-order list of 4 partitions,
-		// which should be sorted before provided to the custom partitioner's open() method
-		int[] correctPartitionList = {0, 1, 2, 3};
-		verify(mockPartitioner).open(0, 1, correctPartitionList);
-	}
-
-	/**
-	 * Test ensuring that the producer is not dropping buffered records.;
-	 * we set a timeout because the test will not finish if the logic is broken
-	 */
-	@Test(timeout=5000)
-	public void testAtLeastOnceProducer() throws Throwable {
-		runAtLeastOnceTest(true);
-	}
-
-	/**
-	 * Ensures that the at least once producing test fails if the flushing is disabled
-	 */
-	@Test(expected = AssertionError.class, timeout=5000)
-	public void testAtLeastOnceProducerFailsIfFlushingDisabled() throws Throwable {
-		runAtLeastOnceTest(false);
-	}
-
-	private void runAtLeastOnceTest(boolean flushOnCheckpoint) throws Throwable {
-		final AtomicBoolean snapshottingFinished = new AtomicBoolean(false);
-		final DummyFlinkKafkaProducer<String> producer = new DummyFlinkKafkaProducer<>(
-			FakeStandardProducerConfig.get(), null, snapshottingFinished);
-		producer.setFlushOnCheckpoint(flushOnCheckpoint);
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness =
-				new OneInputStreamOperatorTestHarness<>(new StreamSink(producer));
-
-		testHarness.open();
-
-		for (int i = 0; i < 100; i++) {
-			testHarness.processElement(new StreamRecord<>("msg-" + i));
-		}
-
-		// start a thread confirming all pending records
-		final Tuple1<Throwable> runnableError = new Tuple1<>(null);
-		final Thread threadA = Thread.currentThread();
-
-		Runnable confirmer = new Runnable() {
-			@Override
-			public void run() {
-				try {
-					MockProducer mp = producer.getProducerInstance();
-					List<Callback> pending = mp.getPending();
-
-					// we need to find out if the snapshot() method blocks forever
-					// this is not possible. If snapshot() is running, it will
-					// start removing elements from the pending list.
-					synchronized (threadA) {
-						threadA.wait(500L);
-					}
-					// we now check that no records have been confirmed yet
-					Assert.assertEquals(100, pending.size());
-					Assert.assertFalse("Snapshot method returned before all records were confirmed",
-						snapshottingFinished.get());
-
-					// now confirm all checkpoints
-					for (Callback c: pending) {
-						c.onCompletion(null, null);
-					}
-					pending.clear();
-				} catch(Throwable t) {
-					runnableError.f0 = t;
-				}
-			}
-		};
-		Thread threadB = new Thread(confirmer);
-		threadB.start();
-
-		// this should block:
-		testHarness.snapshot(0, 0);
-
-		synchronized (threadA) {
-			threadA.notifyAll(); // just in case, to let the test fail faster
-		}
-		Assert.assertEquals(0, producer.getProducerInstance().getPending().size());
-		Deadline deadline = FiniteDuration.apply(5, "s").fromNow();
-		while (deadline.hasTimeLeft() && threadB.isAlive()) {
-			threadB.join(500);
-		}
-		Assert.assertFalse("Thread A is expected to be finished at this point. If not, the test is prone to fail", threadB.isAlive());
-		if (runnableError.f0 != null) {
-			throw runnableError.f0;
-		}
-
-		testHarness.close();
-	}
-
-
-	// ------------------------------------------------------------------------
-
-	private static class DummyFlinkKafkaProducer<T> extends FlinkKafkaProducerBase<T> {
-		private static final long serialVersionUID = 1L;
-
-		private transient MockProducer prod;
-		private AtomicBoolean snapshottingFinished;
-
-		@SuppressWarnings("unchecked")
-		public DummyFlinkKafkaProducer(Properties producerConfig, KafkaPartitioner partitioner, AtomicBoolean snapshottingFinished) {
-			super("dummy-topic", (KeyedSerializationSchema< T >) mock(KeyedSerializationSchema.class), producerConfig, partitioner);
-			this.snapshottingFinished = snapshottingFinished;
-		}
-
-		// constructor variant for test irrelated to snapshotting
-		@SuppressWarnings("unchecked")
-		public DummyFlinkKafkaProducer(Properties producerConfig, KafkaPartitioner partitioner) {
-			super("dummy-topic", (KeyedSerializationSchema< T >) mock(KeyedSerializationSchema.class), producerConfig, partitioner);
-			this.snapshottingFinished = new AtomicBoolean(true);
-		}
-
-		@Override
-		protected <K, V> KafkaProducer<K, V> getKafkaProducer(Properties props) {
-			this.prod = new MockProducer();
-			return this.prod;
-		}
-
-		@Override
-		public void snapshotState(FunctionSnapshotContext ctx) throws Exception {
-			// call the actual snapshot state
-			super.snapshotState(ctx);
-			// notify test that snapshotting has been done
-			snapshottingFinished.set(true);
-		}
-
-		@Override
-		protected void flush() {
-			this.prod.flush();
-		}
-
-		public MockProducer getProducerInstance() {
-			return this.prod;
-		}
-	}
-
-	private static class MockProducer<K, V> extends KafkaProducer<K, V> {
-		List<Callback> pendingCallbacks = new ArrayList<>();
-
-		public MockProducer() {
-			super(FakeStandardProducerConfig.get());
-		}
-
-		@Override
-		public Future<RecordMetadata> send(ProducerRecord<K, V> record) {
-			throw new UnsupportedOperationException("Unexpected");
-		}
-
-		@Override
-		public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
-			pendingCallbacks.add(callback);
-			return null;
-		}
-
-		@Override
-		public List<PartitionInfo> partitionsFor(String topic) {
-			List<PartitionInfo> list = new ArrayList<>();
-			// deliberately return an out-of-order partition list
-			list.add(new PartitionInfo(topic, 3, null, null, null));
-			list.add(new PartitionInfo(topic, 1, null, null, null));
-			list.add(new PartitionInfo(topic, 0, null, null, null));
-			list.add(new PartitionInfo(topic, 2, null, null, null));
-			return list;
-		}
-
-		@Override
-		public Map<MetricName, ? extends Metric> metrics() {
-			return null;
-		}
-
-
-		public List<Callback> getPending() {
-			return this.pendingCallbacks;
-		}
-
-		public void flush() {
-			while (pendingCallbacks.size() > 0) {
-				try {
-					Thread.sleep(10);
-				} catch (InterruptedException e) {
-					throw new RuntimeException("Unable to flush producer, task was interrupted");
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java
deleted file mode 100644
index 1882a7e..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.streaming.util.serialization.JSONDeserializationSchema;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-
-public class JSONDeserializationSchemaTest {
-	@Test
-	public void testDeserialize() throws IOException {
-		ObjectMapper mapper = new ObjectMapper();
-		ObjectNode initialValue = mapper.createObjectNode();
-		initialValue.put("key", 4).put("value", "world");
-		byte[] serializedValue = mapper.writeValueAsBytes(initialValue);
-
-		JSONDeserializationSchema schema = new JSONDeserializationSchema();
-		ObjectNode deserializedValue = schema.deserialize(serializedValue);
-
-		Assert.assertEquals(4, deserializedValue.get("key").asInt());
-		Assert.assertEquals("world", deserializedValue.get("value").asText());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java
deleted file mode 100644
index 86d3105..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-
-public class JSONKeyValueDeserializationSchemaTest {
-	@Test
-	public void testDeserializeWithoutMetadata() throws IOException {
-		ObjectMapper mapper = new ObjectMapper();
-		ObjectNode initialKey = mapper.createObjectNode();
-		initialKey.put("index", 4);
-		byte[] serializedKey = mapper.writeValueAsBytes(initialKey);
-
-		ObjectNode initialValue = mapper.createObjectNode();
-		initialValue.put("word", "world");
-		byte[] serializedValue = mapper.writeValueAsBytes(initialValue);
-
-		JSONKeyValueDeserializationSchema schema = new JSONKeyValueDeserializationSchema(false);
-		ObjectNode deserializedValue = schema.deserialize(serializedKey, serializedValue, "", 0, 0);
-
-
-		Assert.assertTrue(deserializedValue.get("metadata") == null);
-		Assert.assertEquals(4, deserializedValue.get("key").get("index").asInt());
-		Assert.assertEquals("world", deserializedValue.get("value").get("word").asText());
-	}
-
-	@Test
-	public void testDeserializeWithMetadata() throws IOException {
-		ObjectMapper mapper = new ObjectMapper();
-		ObjectNode initialKey = mapper.createObjectNode();
-		initialKey.put("index", 4);
-		byte[] serializedKey = mapper.writeValueAsBytes(initialKey);
-
-		ObjectNode initialValue = mapper.createObjectNode();
-		initialValue.put("word", "world");
-		byte[] serializedValue = mapper.writeValueAsBytes(initialValue);
-
-		JSONKeyValueDeserializationSchema schema = new JSONKeyValueDeserializationSchema(true);
-		ObjectNode deserializedValue = schema.deserialize(serializedKey, serializedValue, "topic#1", 3, 4);
-
-		Assert.assertEquals(4, deserializedValue.get("key").get("index").asInt());
-		Assert.assertEquals("world", deserializedValue.get("value").get("word").asText());
-		Assert.assertEquals("topic#1", deserializedValue.get("metadata").get("topic").asText());
-		Assert.assertEquals(4, deserializedValue.get("metadata").get("offset").asInt());
-		Assert.assertEquals(3, deserializedValue.get("metadata").get("partition").asInt());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java
deleted file mode 100644
index 68225e2..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class JsonRowDeserializationSchemaTest {
-
-	/**
-	 * Tests simple deserialization.
-	 */
-	@Test
-	public void testDeserialization() throws Exception {
-		long id = 1238123899121L;
-		String name = "asdlkjasjkdla998y1122";
-		byte[] bytes = new byte[1024];
-		ThreadLocalRandom.current().nextBytes(bytes);
-
-		ObjectMapper objectMapper = new ObjectMapper();
-
-		// Root
-		ObjectNode root = objectMapper.createObjectNode();
-		root.put("id", id);
-		root.put("name", name);
-		root.put("bytes", bytes);
-
-		byte[] serializedJson = objectMapper.writeValueAsBytes(root);
-
-		JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(
-				new String[] { "id", "name", "bytes" },
-				new Class<?>[] { Long.class, String.class, byte[].class });
-
-		Row deserialized = deserializationSchema.deserialize(serializedJson);
-
-		assertEquals(3, deserialized.productArity());
-		assertEquals(id, deserialized.productElement(0));
-		assertEquals(name, deserialized.productElement(1));
-		assertArrayEquals(bytes, (byte[]) deserialized.productElement(2));
-	}
-
-	/**
-	 * Tests deserialization with non-existing field name.
-	 */
-	@Test
-	public void testMissingNode() throws Exception {
-		ObjectMapper objectMapper = new ObjectMapper();
-
-		// Root
-		ObjectNode root = objectMapper.createObjectNode();
-		root.put("id", 123123123);
-		byte[] serializedJson = objectMapper.writeValueAsBytes(root);
-
-		JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(
-				new String[] { "name" },
-				new Class<?>[] { String.class });
-
-		Row row = deserializationSchema.deserialize(serializedJson);
-
-		assertEquals(1, row.productArity());
-		assertNull("Missing field not null", row.productElement(0));
-
-		deserializationSchema.setFailOnMissingField(true);
-
-		try {
-			deserializationSchema.deserialize(serializedJson);
-			fail("Did not throw expected Exception");
-		} catch (IOException e) {
-			assertTrue(e.getCause() instanceof IllegalStateException);
-		}
-	}
-
-	/**
-	 * Tests that number of field names and types has to match.
-	 */
-	@Test
-	public void testNumberOfFieldNamesAndTypesMismatch() throws Exception {
-		try {
-			new JsonRowDeserializationSchema(
-					new String[] { "one", "two", "three" },
-					new Class<?>[] { Long.class });
-			fail("Did not throw expected Exception");
-		} catch (IllegalArgumentException ignored) {
-			// Expected
-		}
-
-		try {
-			new JsonRowDeserializationSchema(
-					new String[] { "one" },
-					new Class<?>[] { Long.class, String.class });
-			fail("Did not throw expected Exception");
-		} catch (IllegalArgumentException ignored) {
-			// Expected
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java
deleted file mode 100644
index 92af15d..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-public class JsonRowSerializationSchemaTest {
-	@Test
-	public void testRowSerialization() throws IOException {
-		String[] fieldNames = new String[] {"f1", "f2", "f3"};
-		Class[] fieldTypes = new Class[] {Integer.class, Boolean.class, String.class};
-		Row row = new Row(3);
-		row.setField(0, 1);
-		row.setField(1, true);
-		row.setField(2, "str");
-
-		Row resultRow = serializeAndDeserialize(fieldNames, fieldTypes, row);
-		assertEqualRows(row, resultRow);
-	}
-
-	@Test
-	public void testSerializationOfTwoRows() throws IOException {
-		String[] fieldNames = new String[] {"f1", "f2", "f3"};
-		Class[] fieldTypes = new Class[] {Integer.class, Boolean.class, String.class};
-		Row row1 = new Row(3);
-		row1.setField(0, 1);
-		row1.setField(1, true);
-		row1.setField(2, "str");
-
-		JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(fieldNames);
-		JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(fieldNames, fieldTypes);
-
-		byte[] bytes = serializationSchema.serialize(row1);
-		assertEqualRows(row1, deserializationSchema.deserialize(bytes));
-
-		Row row2 = new Row(3);
-		row2.setField(0, 10);
-		row2.setField(1, false);
-		row2.setField(2, "newStr");
-
-		bytes = serializationSchema.serialize(row2);
-		assertEqualRows(row2, deserializationSchema.deserialize(bytes));
-	}
-
-	@Test(expected = NullPointerException.class)
-	public void testInputValidation() {
-		new JsonRowSerializationSchema(null);
-	}
-
-	@Test(expected = IllegalStateException.class)
-	public void testSerializeRowWithInvalidNumberOfFields() {
-		String[] fieldNames = new String[] {"f1", "f2", "f3"};
-		Row row = new Row(1);
-		row.setField(0, 1);
-
-		JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(fieldNames);
-		serializationSchema.serialize(row);
-	}
-
-	private Row serializeAndDeserialize(String[] fieldNames, Class[] fieldTypes, Row row) throws IOException {
-		JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(fieldNames);
-		JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(fieldNames, fieldTypes);
-
-		byte[] bytes = serializationSchema.serialize(row);
-		return deserializationSchema.deserialize(bytes);
-	}
-
-	private void assertEqualRows(Row expectedRow, Row resultRow) {
-		assertEquals("Deserialized row should have expected number of fields",
-			expectedRow.productArity(), resultRow.productArity());
-		for (int i = 0; i < expectedRow.productArity(); i++) {
-			assertEquals(String.format("Field number %d should be as in the original row", i),
-				expectedRow.productElement(i), resultRow.productElement(i));
-		}
-	}
-
-}

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


[44/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroSplittableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroSplittableInputFormatTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroSplittableInputFormatTest.java
new file mode 100644
index 0000000..37a83d1
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroSplittableInputFormatTest.java
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.avro;
+
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.flink.api.io.avro.generated.Address;
+import org.apache.flink.api.io.avro.generated.Colors;
+import org.apache.flink.api.io.avro.generated.Fixed16;
+import org.apache.flink.api.io.avro.generated.User;
+import org.apache.flink.api.java.io.AvroInputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test the avro input format.
+ * (The testcase is mostly the getting started tutorial of avro)
+ * http://avro.apache.org/docs/current/gettingstartedjava.html
+ */
+public class AvroSplittableInputFormatTest {
+	
+	private File testFile;
+	
+	final static String TEST_NAME = "Alyssa";
+	
+	final static String TEST_ARRAY_STRING_1 = "ELEMENT 1";
+	final static String TEST_ARRAY_STRING_2 = "ELEMENT 2";
+	
+	final static boolean TEST_ARRAY_BOOLEAN_1 = true;
+	final static boolean TEST_ARRAY_BOOLEAN_2 = false;
+	
+	final static Colors TEST_ENUM_COLOR = Colors.GREEN;
+	
+	final static String TEST_MAP_KEY1 = "KEY 1";
+	final static long TEST_MAP_VALUE1 = 8546456L;
+	final static String TEST_MAP_KEY2 = "KEY 2";
+	final static long TEST_MAP_VALUE2 = 17554L;
+
+	final static Integer TEST_NUM = new Integer(239);
+	final static String TEST_STREET = "Baker Street";
+	final static String TEST_CITY = "London";
+	final static String TEST_STATE = "London";
+	final static String TEST_ZIP = "NW1 6XE";
+	
+	final static int NUM_RECORDS = 5000;
+
+	@Before
+	public void createFiles() throws IOException {
+		testFile = File.createTempFile("AvroSplittableInputFormatTest", null);
+		
+		ArrayList<CharSequence> stringArray = new ArrayList<CharSequence>();
+		stringArray.add(TEST_ARRAY_STRING_1);
+		stringArray.add(TEST_ARRAY_STRING_2);
+		
+		ArrayList<Boolean> booleanArray = new ArrayList<Boolean>();
+		booleanArray.add(TEST_ARRAY_BOOLEAN_1);
+		booleanArray.add(TEST_ARRAY_BOOLEAN_2);
+		
+		HashMap<CharSequence, Long> longMap = new HashMap<CharSequence, Long>();
+		longMap.put(TEST_MAP_KEY1, TEST_MAP_VALUE1);
+		longMap.put(TEST_MAP_KEY2, TEST_MAP_VALUE2);
+		
+		Address addr = new Address();
+		addr.setNum(new Integer(TEST_NUM));
+		addr.setStreet(TEST_STREET);
+		addr.setCity(TEST_CITY);
+		addr.setState(TEST_STATE);
+		addr.setZip(TEST_ZIP);
+		
+		
+		User user1 = new User();
+		user1.setName(TEST_NAME);
+		user1.setFavoriteNumber(256);
+		user1.setTypeDoubleTest(123.45d);
+		user1.setTypeBoolTest(true);
+		user1.setTypeArrayString(stringArray);
+		user1.setTypeArrayBoolean(booleanArray);
+		user1.setTypeEnum(TEST_ENUM_COLOR);
+		user1.setTypeMap(longMap);
+		user1.setTypeNested(addr);
+		
+		// Construct via builder
+		User user2 = User.newBuilder()
+		             .setName(TEST_NAME)
+		             .setFavoriteColor("blue")
+		             .setFavoriteNumber(null)
+		             .setTypeBoolTest(false)
+		             .setTypeDoubleTest(1.337d)
+		             .setTypeNullTest(null)
+		             .setTypeLongTest(1337L)
+		             .setTypeArrayString(new ArrayList<CharSequence>())
+		             .setTypeArrayBoolean(new ArrayList<Boolean>())
+		             .setTypeNullableArray(null)
+		             .setTypeEnum(Colors.RED)
+		             .setTypeMap(new HashMap<CharSequence, Long>())
+					 .setTypeFixed(new Fixed16())
+					 .setTypeUnion(123L)
+				.setTypeNested(
+						Address.newBuilder().setNum(TEST_NUM).setStreet(TEST_STREET)
+								.setCity(TEST_CITY).setState(TEST_STATE).setZip(TEST_ZIP)
+								.build())
+
+		             .build();
+		DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
+		DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
+		dataFileWriter.create(user1.getSchema(), testFile);
+		dataFileWriter.append(user1);
+		dataFileWriter.append(user2);
+
+		Random rnd = new Random(1337);
+		for(int i = 0; i < NUM_RECORDS -2 ; i++) {
+			User user = new User();
+			user.setName(TEST_NAME + rnd.nextInt());
+			user.setFavoriteNumber(rnd.nextInt());
+			user.setTypeDoubleTest(rnd.nextDouble());
+			user.setTypeBoolTest(true);
+			user.setTypeArrayString(stringArray);
+			user.setTypeArrayBoolean(booleanArray);
+			user.setTypeEnum(TEST_ENUM_COLOR);
+			user.setTypeMap(longMap);
+			Address address = new Address();
+			address.setNum(new Integer(TEST_NUM));
+			address.setStreet(TEST_STREET);
+			address.setCity(TEST_CITY);
+			address.setState(TEST_STATE);
+			address.setZip(TEST_ZIP);
+			user.setTypeNested(address);
+
+			dataFileWriter.append(user);
+		}
+		dataFileWriter.close();
+	}
+	
+	@Test
+	public void testSplittedIF() throws IOException {
+		Configuration parameters = new Configuration();
+		
+		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
+
+		format.configure(parameters);
+		FileInputSplit[] splits = format.createInputSplits(4);
+		assertEquals(splits.length, 4);
+		int elements = 0;
+		int elementsPerSplit[] = new int[4];
+		for(int i = 0; i < splits.length; i++) {
+			format.open(splits[i]);
+			while(!format.reachedEnd()) {
+				User u = format.nextRecord(null);
+				Assert.assertTrue(u.getName().toString().startsWith(TEST_NAME));
+				elements++;
+				elementsPerSplit[i]++;
+			}
+			format.close();
+		}
+
+		Assert.assertEquals(1539, elementsPerSplit[0]);
+		Assert.assertEquals(1026, elementsPerSplit[1]);
+		Assert.assertEquals(1539, elementsPerSplit[2]);
+		Assert.assertEquals(896, elementsPerSplit[3]);
+		Assert.assertEquals(NUM_RECORDS, elements);
+		format.close();
+	}
+
+	@Test
+	public void testAvroRecoveryWithFailureAtStart() throws Exception {
+		final int recordsUntilCheckpoint = 132;
+
+		Configuration parameters = new Configuration();
+
+		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
+		format.configure(parameters);
+
+		FileInputSplit[] splits = format.createInputSplits(4);
+		assertEquals(splits.length, 4);
+
+		int elements = 0;
+		int elementsPerSplit[] = new int[4];
+		for(int i = 0; i < splits.length; i++) {
+			format.reopen(splits[i], format.getCurrentState());
+			while(!format.reachedEnd()) {
+				User u = format.nextRecord(null);
+				Assert.assertTrue(u.getName().toString().startsWith(TEST_NAME));
+				elements++;
+
+				if(format.getRecordsReadFromBlock() == recordsUntilCheckpoint) {
+
+					// do the whole checkpoint-restore procedure and see if we pick up from where we left off.
+					Tuple2<Long, Long> state = format.getCurrentState();
+
+					// this is to make sure that nothing stays from the previous format
+					// (as it is going to be in the normal case)
+					format = new AvroInputFormat<>(new Path(testFile.getAbsolutePath()), User.class);
+
+					format.reopen(splits[i], state);
+					assertEquals(format.getRecordsReadFromBlock(), recordsUntilCheckpoint);
+				}
+				elementsPerSplit[i]++;
+			}
+			format.close();
+		}
+
+		Assert.assertEquals(1539, elementsPerSplit[0]);
+		Assert.assertEquals(1026, elementsPerSplit[1]);
+		Assert.assertEquals(1539, elementsPerSplit[2]);
+		Assert.assertEquals(896, elementsPerSplit[3]);
+		Assert.assertEquals(NUM_RECORDS, elements);
+		format.close();
+	}
+
+	@Test
+	public void testAvroRecovery() throws Exception {
+		final int recordsUntilCheckpoint = 132;
+
+		Configuration parameters = new Configuration();
+
+		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
+		format.configure(parameters);
+
+		FileInputSplit[] splits = format.createInputSplits(4);
+		assertEquals(splits.length, 4);
+
+		int elements = 0;
+		int elementsPerSplit[] = new int[4];
+		for(int i = 0; i < splits.length; i++) {
+			format.open(splits[i]);
+			while(!format.reachedEnd()) {
+				User u = format.nextRecord(null);
+				Assert.assertTrue(u.getName().toString().startsWith(TEST_NAME));
+				elements++;
+
+				if(format.getRecordsReadFromBlock() == recordsUntilCheckpoint) {
+
+					// do the whole checkpoint-restore procedure and see if we pick up from where we left off.
+					Tuple2<Long, Long> state = format.getCurrentState();
+
+					// this is to make sure that nothing stays from the previous format
+					// (as it is going to be in the normal case)
+					format = new AvroInputFormat<>(new Path(testFile.getAbsolutePath()), User.class);
+
+					format.reopen(splits[i], state);
+					assertEquals(format.getRecordsReadFromBlock(), recordsUntilCheckpoint);
+				}
+				elementsPerSplit[i]++;
+			}
+			format.close();
+		}
+
+		Assert.assertEquals(1539, elementsPerSplit[0]);
+		Assert.assertEquals(1026, elementsPerSplit[1]);
+		Assert.assertEquals(1539, elementsPerSplit[2]);
+		Assert.assertEquals(896, elementsPerSplit[3]);
+		Assert.assertEquals(NUM_RECORDS, elements);
+		format.close();
+	}
+
+	/*
+	This test is gave the reference values for the test of Flink's IF.
+
+	This dependency needs to be added
+
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro-mapred</artifactId>
+            <version>1.7.6</version>
+        </dependency>
+
+	@Test
+	public void testHadoop() throws Exception {
+		JobConf jf = new JobConf();
+		FileInputFormat.addInputPath(jf, new org.apache.hadoop.fs.Path(testFile.toURI()));
+		jf.setBoolean(org.apache.avro.mapred.AvroInputFormat.IGNORE_FILES_WITHOUT_EXTENSION_KEY, false);
+		org.apache.avro.mapred.AvroInputFormat<User> format = new org.apache.avro.mapred.AvroInputFormat<User>();
+		InputSplit[] sp = format.getSplits(jf, 4);
+		int elementsPerSplit[] = new int[4];
+		int cnt = 0;
+		int i = 0;
+		for(InputSplit s:sp) {
+			RecordReader<AvroWrapper<User>, NullWritable> r = format.getRecordReader(s, jf, new HadoopDummyReporter());
+			AvroWrapper<User> k = r.createKey();
+			NullWritable v = r.createValue();
+
+			while(r.next(k,v)) {
+				cnt++;
+				elementsPerSplit[i]++;
+			}
+			i++;
+		}
+		System.out.println("Status "+Arrays.toString(elementsPerSplit));
+	} **/
+	
+	@After
+	public void deleteFiles() {
+		testFile.delete();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/AvroTypeExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/AvroTypeExample.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/AvroTypeExample.java
new file mode 100644
index 0000000..5a21691
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/AvroTypeExample.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.io.avro.example;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.io.GenericInputFormat;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+
+@SuppressWarnings("serial")
+public class AvroTypeExample {
+	
+	
+	public static void main(String[] args) throws Exception {
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<User> users = env.createInput(new UserGeneratingInputFormat());
+
+		users
+			.map(new NumberExtractingMapper())
+			.groupBy(1)
+			.reduceGroup(new ConcatenatingReducer())
+			.print();
+	}
+	
+	
+	public static final class NumberExtractingMapper implements MapFunction<User, Tuple2<User, Integer>> {
+		
+		@Override
+		public Tuple2<User, Integer> map(User user) {
+			return new Tuple2<User, Integer>(user, user.getFavoriteNumber());
+		}
+	}
+	
+	
+	public static final class ConcatenatingReducer implements GroupReduceFunction<Tuple2<User, Integer>, Tuple2<Integer, String>> {
+
+		@Override
+		public void reduce(Iterable<Tuple2<User, Integer>> values, Collector<Tuple2<Integer, String>> out) throws Exception {
+			int number = 0;
+			StringBuilder colors = new StringBuilder();
+			
+			for (Tuple2<User, Integer> u : values) {
+				number = u.f1;
+				colors.append(u.f0.getFavoriteColor()).append(" - ");
+			}
+			
+			colors.setLength(colors.length() - 3);
+			out.collect(new Tuple2<Integer, String>(number, colors.toString()));
+		}
+	}
+	
+	
+	public static final class UserGeneratingInputFormat extends GenericInputFormat<User> {
+
+		private static final long serialVersionUID = 1L;
+		
+		private static final int NUM = 100;
+		
+		private final Random rnd = new Random(32498562304986L);
+		
+		private static final String[] NAMES = { "Peter", "Bob", "Liddy", "Alexander", "Stan" };
+		
+		private static final String[] COLORS = { "mauve", "crimson", "copper", "sky", "grass" };
+		
+		private int count;
+		
+
+		@Override
+		public boolean reachedEnd() throws IOException {
+			return count >= NUM;
+		}
+
+		@Override
+		public User nextRecord(User reuse) throws IOException {
+			count++;
+			
+			User u = new User();
+			u.setName(NAMES[rnd.nextInt(NAMES.length)]);
+			u.setFavoriteColor(COLORS[rnd.nextInt(COLORS.length)]);
+			u.setFavoriteNumber(rnd.nextInt(87));
+			return u;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/User.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/User.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/User.java
new file mode 100644
index 0000000..4608f96
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/User.java
@@ -0,0 +1,269 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+/**
+ * Autogenerated by Avro
+ * 
+ * DO NOT EDIT DIRECTLY
+ */
+package org.apache.flink.api.io.avro.example;  
+@SuppressWarnings("all")
+@org.apache.avro.specific.AvroGenerated
+public class User extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
+  public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"User\",\"namespace\":\"org.apache.flink.api.avro.example\",\"fields\":[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"favorite_number\",\"type\":[\"int\",\"null\"]},{\"name\":\"favorite_color\",\"type\":[\"string\",\"null\"]}]}");
+  public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; }
+  @Deprecated public java.lang.CharSequence name;
+  @Deprecated public java.lang.Integer favorite_number;
+  @Deprecated public java.lang.CharSequence favorite_color;
+
+  /**
+   * Default constructor.  Note that this does not initialize fields
+   * to their default values from the schema.  If that is desired then
+   * one should use {@link #newBuilder()}. 
+   */
+  public User() {}
+
+  /**
+   * All-args constructor.
+   */
+  public User(java.lang.CharSequence name, java.lang.Integer favorite_number, java.lang.CharSequence favorite_color) {
+    this.name = name;
+    this.favorite_number = favorite_number;
+    this.favorite_color = favorite_color;
+  }
+
+  public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+  // Used by DatumWriter.  Applications should not call. 
+  public java.lang.Object get(int field$) {
+    switch (field$) {
+    case 0: return name;
+    case 1: return favorite_number;
+    case 2: return favorite_color;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+  // Used by DatumReader.  Applications should not call. 
+  @SuppressWarnings(value="unchecked")
+  public void put(int field$, java.lang.Object value$) {
+    switch (field$) {
+    case 0: name = (java.lang.CharSequence)value$; break;
+    case 1: favorite_number = (java.lang.Integer)value$; break;
+    case 2: favorite_color = (java.lang.CharSequence)value$; break;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+
+  /**
+   * Gets the value of the 'name' field.
+   */
+  public java.lang.CharSequence getName() {
+    return name;
+  }
+
+  /**
+   * Sets the value of the 'name' field.
+   * @param value the value to set.
+   */
+  public void setName(java.lang.CharSequence value) {
+    this.name = value;
+  }
+
+  /**
+   * Gets the value of the 'favorite_number' field.
+   */
+  public java.lang.Integer getFavoriteNumber() {
+    return favorite_number;
+  }
+
+  /**
+   * Sets the value of the 'favorite_number' field.
+   * @param value the value to set.
+   */
+  public void setFavoriteNumber(java.lang.Integer value) {
+    this.favorite_number = value;
+  }
+
+  /**
+   * Gets the value of the 'favorite_color' field.
+   */
+  public java.lang.CharSequence getFavoriteColor() {
+    return favorite_color;
+  }
+
+  /**
+   * Sets the value of the 'favorite_color' field.
+   * @param value the value to set.
+   */
+  public void setFavoriteColor(java.lang.CharSequence value) {
+    this.favorite_color = value;
+  }
+
+  /** Creates a new User RecordBuilder */
+  public static org.apache.flink.api.io.avro.example.User.Builder newBuilder() {
+    return new org.apache.flink.api.io.avro.example.User.Builder();
+  }
+  
+  /** Creates a new User RecordBuilder by copying an existing Builder */
+  public static org.apache.flink.api.io.avro.example.User.Builder newBuilder(org.apache.flink.api.io.avro.example.User.Builder other) {
+    return new org.apache.flink.api.io.avro.example.User.Builder(other);
+  }
+  
+  /** Creates a new User RecordBuilder by copying an existing User instance */
+  public static org.apache.flink.api.io.avro.example.User.Builder newBuilder(org.apache.flink.api.io.avro.example.User other) {
+    return new org.apache.flink.api.io.avro.example.User.Builder(other);
+  }
+  
+  /**
+   * RecordBuilder for User instances.
+   */
+  public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase<User>
+    implements org.apache.avro.data.RecordBuilder<User> {
+
+    private java.lang.CharSequence name;
+    private java.lang.Integer favorite_number;
+    private java.lang.CharSequence favorite_color;
+
+    /** Creates a new Builder */
+    private Builder() {
+      super(org.apache.flink.api.io.avro.example.User.SCHEMA$);
+    }
+    
+    /** Creates a Builder by copying an existing Builder */
+    private Builder(org.apache.flink.api.io.avro.example.User.Builder other) {
+      super(other);
+      if (isValidValue(fields()[0], other.name)) {
+        this.name = data().deepCopy(fields()[0].schema(), other.name);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.favorite_number)) {
+        this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.favorite_color)) {
+        this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color);
+        fieldSetFlags()[2] = true;
+      }
+    }
+    
+    /** Creates a Builder by copying an existing User instance */
+    private Builder(org.apache.flink.api.io.avro.example.User other) {
+            super(org.apache.flink.api.io.avro.example.User.SCHEMA$);
+      if (isValidValue(fields()[0], other.name)) {
+        this.name = data().deepCopy(fields()[0].schema(), other.name);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.favorite_number)) {
+        this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.favorite_color)) {
+        this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color);
+        fieldSetFlags()[2] = true;
+      }
+    }
+
+    /** Gets the value of the 'name' field */
+    public java.lang.CharSequence getName() {
+      return name;
+    }
+    
+    /** Sets the value of the 'name' field */
+    public org.apache.flink.api.io.avro.example.User.Builder setName(java.lang.CharSequence value) {
+      validate(fields()[0], value);
+      this.name = value;
+      fieldSetFlags()[0] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'name' field has been set */
+    public boolean hasName() {
+      return fieldSetFlags()[0];
+    }
+    
+    /** Clears the value of the 'name' field */
+    public org.apache.flink.api.io.avro.example.User.Builder clearName() {
+      name = null;
+      fieldSetFlags()[0] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'favorite_number' field */
+    public java.lang.Integer getFavoriteNumber() {
+      return favorite_number;
+    }
+    
+    /** Sets the value of the 'favorite_number' field */
+    public org.apache.flink.api.io.avro.example.User.Builder setFavoriteNumber(java.lang.Integer value) {
+      validate(fields()[1], value);
+      this.favorite_number = value;
+      fieldSetFlags()[1] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'favorite_number' field has been set */
+    public boolean hasFavoriteNumber() {
+      return fieldSetFlags()[1];
+    }
+    
+    /** Clears the value of the 'favorite_number' field */
+    public org.apache.flink.api.io.avro.example.User.Builder clearFavoriteNumber() {
+      favorite_number = null;
+      fieldSetFlags()[1] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'favorite_color' field */
+    public java.lang.CharSequence getFavoriteColor() {
+      return favorite_color;
+    }
+    
+    /** Sets the value of the 'favorite_color' field */
+    public org.apache.flink.api.io.avro.example.User.Builder setFavoriteColor(java.lang.CharSequence value) {
+      validate(fields()[2], value);
+      this.favorite_color = value;
+      fieldSetFlags()[2] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'favorite_color' field has been set */
+    public boolean hasFavoriteColor() {
+      return fieldSetFlags()[2];
+    }
+    
+    /** Clears the value of the 'favorite_color' field */
+    public org.apache.flink.api.io.avro.example.User.Builder clearFavoriteColor() {
+      favorite_color = null;
+      fieldSetFlags()[2] = false;
+      return this;
+    }
+
+    @Override
+    public User build() {
+      try {
+        User record = new User();
+        record.name = fieldSetFlags()[0] ? this.name : (java.lang.CharSequence) defaultValue(fields()[0]);
+        record.favorite_number = fieldSetFlags()[1] ? this.favorite_number : (java.lang.Integer) defaultValue(fields()[1]);
+        record.favorite_color = fieldSetFlags()[2] ? this.favorite_color : (java.lang.CharSequence) defaultValue(fields()[2]);
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroInputFormatTypeExtractionTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroInputFormatTypeExtractionTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroInputFormatTypeExtractionTest.java
new file mode 100644
index 0000000..e245026
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroInputFormatTypeExtractionTest.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.api.java.io;
+
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.typeutils.PojoTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.fs.Path;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class AvroInputFormatTypeExtractionTest {
+
+	@Test
+	public void testTypeExtraction() {
+		try {
+			InputFormat<MyAvroType, ?> format = new AvroInputFormat<MyAvroType>(new Path("file:///ignore/this/file"), MyAvroType.class);
+
+			TypeInformation<?> typeInfoDirect = TypeExtractor.getInputFormatTypes(format);
+
+			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+			DataSet<MyAvroType> input = env.createInput(format);
+			TypeInformation<?> typeInfoDataSet = input.getType();
+
+
+			Assert.assertTrue(typeInfoDirect instanceof PojoTypeInfo);
+			Assert.assertTrue(typeInfoDataSet instanceof PojoTypeInfo);
+
+			Assert.assertEquals(MyAvroType.class, typeInfoDirect.getTypeClass());
+			Assert.assertEquals(MyAvroType.class, typeInfoDataSet.getTypeClass());
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail(e.getMessage());
+		}
+	}
+
+	public static final class MyAvroType {
+
+		public String theString;
+
+		public MyAvroType recursive;
+
+		private double aDouble;
+
+		public double getaDouble() {
+			return aDouble;
+		}
+
+		public void setaDouble(double aDouble) {
+			this.aDouble = aDouble;
+		}
+
+		public void setTheString(String theString) {
+			this.theString = theString;
+		}
+
+		public String getTheString() {
+			return theString;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java
new file mode 100644
index 0000000..4d6c6b7
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 static org.apache.flink.api.java.io.AvroOutputFormat.Codec;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+import org.apache.avro.Schema;
+import org.apache.flink.api.io.avro.example.User;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+/**
+ * Tests for {@link AvroOutputFormat}
+ */
+public class AvroOutputFormatTest {
+
+	@Test
+	public void testSetCodec() throws Exception {
+		// given
+		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
+
+		// when
+		try {
+			outputFormat.setCodec(Codec.SNAPPY);
+		} catch (Exception ex) {
+			// then
+			fail("unexpected exception");
+		}
+	}
+
+	@Test
+	public void testSetCodecError() throws Exception {
+		// given
+		boolean error = false;
+		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
+
+		// when
+		try {
+			outputFormat.setCodec(null);
+		} catch (Exception ex) {
+			error = true;
+		}
+
+		// then
+		assertTrue(error);
+	}
+
+	@Test
+	public void testSerialization() throws Exception {
+
+		serializeAndDeserialize(null, null);
+		serializeAndDeserialize(null, User.SCHEMA$);
+		for (final Codec codec : Codec.values()) {
+			serializeAndDeserialize(codec, null);
+			serializeAndDeserialize(codec, User.SCHEMA$);
+		}
+	}
+
+	private void serializeAndDeserialize(final Codec codec, final Schema schema) throws IOException, ClassNotFoundException {
+		// given
+		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
+		if (codec != null) {
+			outputFormat.setCodec(codec);
+		}
+		if (schema != null) {
+			outputFormat.setSchema(schema);
+		}
+
+		final ByteArrayOutputStream bos = new ByteArrayOutputStream();
+
+		// when
+		try (final ObjectOutputStream oos = new ObjectOutputStream(bos)) {
+			oos.writeObject(outputFormat);
+		}
+		try (final ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(bos.toByteArray()))) {
+			// then
+			Object o = ois.readObject();
+			assertTrue(o instanceof AvroOutputFormat);
+			final AvroOutputFormat<User> restored = (AvroOutputFormat<User>) o;
+			final Codec restoredCodec = (Codec) Whitebox.getInternalState(restored, "codec");
+			final Schema restoredSchema = (Schema) Whitebox.getInternalState(restored, "userDefinedSchema");
+
+			assertTrue(codec != null ? restoredCodec == codec : restoredCodec == null);
+			assertTrue(schema != null ? restoredSchema.equals(schema) : restoredSchema == null);
+		}
+	}
+
+	@Test
+	public void testCompression() throws Exception {
+		// given
+		final Path outputPath = new Path(File.createTempFile("avro-output-file","avro").getAbsolutePath());
+		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(outputPath,User.class);
+		outputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
+
+		final Path compressedOutputPath = new Path(File.createTempFile("avro-output-file","compressed.avro").getAbsolutePath());
+		final AvroOutputFormat<User> compressedOutputFormat = new AvroOutputFormat<>(compressedOutputPath,User.class);
+		compressedOutputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
+		compressedOutputFormat.setCodec(Codec.SNAPPY);
+
+		// when
+		output(outputFormat);
+		output(compressedOutputFormat);
+
+		// then
+		assertTrue(fileSize(outputPath) > fileSize(compressedOutputPath));
+
+		// cleanup
+		Files.delete(Paths.get(outputPath.getPath()));
+		Files.delete(Paths.get(compressedOutputPath.getPath()));
+	}
+
+	private long fileSize(Path path) throws IOException {
+		return Files.size(Paths.get(path.getPath()));
+	}
+
+	private void output(final AvroOutputFormat<User> outputFormat) throws IOException {
+		outputFormat.configure(new Configuration());
+		outputFormat.open(1,1);
+		for (int i = 0; i < 100; i++) {
+			outputFormat.writeRecord(new User("testUser",1,"blue"));
+		}
+		outputFormat.close();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/resources/avro/user.avsc
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/resources/avro/user.avsc b/flink-connectors/flink-avro/src/test/resources/avro/user.avsc
new file mode 100644
index 0000000..02c11af
--- /dev/null
+++ b/flink-connectors/flink-avro/src/test/resources/avro/user.avsc
@@ -0,0 +1,35 @@
+[
+{"namespace": "org.apache.flink.api.io.avro.generated",
+ "type": "record",
+ "name": "Address",
+ "fields": [
+     {"name": "num", "type": "int"},
+     {"name": "street", "type": "string"},
+     {"name": "city", "type": "string"},
+     {"name": "state", "type": "string"},
+     {"name": "zip", "type": "string"}
+  ]
+},
+{"namespace": "org.apache.flink.api.io.avro.generated",
+ "type": "record",
+ "name": "User",
+ "fields": [
+     {"name": "name", "type": "string"},
+     {"name": "favorite_number",  "type": ["int", "null"]},
+     {"name": "favorite_color", "type": ["string", "null"]},
+     {"name": "type_long_test", "type": ["long", "null"]},
+     {"name": "type_double_test", "type": "double"},
+     {"name": "type_null_test", "type": ["null"]},
+     {"name": "type_bool_test", "type": ["boolean"]},
+     {"name": "type_array_string", "type" : {"type" : "array", "items" : "string"}},  
+     {"name": "type_array_boolean", "type" : {"type" : "array", "items" : "boolean"}}, 
+     {"name": "type_nullable_array", "type": ["null", {"type":"array", "items":"string"}], "default":null},
+     {"name": "type_enum", "type": {"type": "enum", "name": "Colors", "symbols" : ["RED", "GREEN", "BLUE"]}},
+     {"name": "type_map", "type": {"type": "map", "values": "long"}},
+     {"name": "type_fixed",
+                 "size": 16,
+                 "type": ["null", {"name": "Fixed16", "size": 16, "type": "fixed"}] },
+     {"name": "type_union", "type": ["null", "boolean", "long", "double"]},
+     {"name": "type_nested", "type": ["null", "Address"]}
+ ]
+}]

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/test/resources/testdata.avro
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/resources/testdata.avro b/flink-connectors/flink-avro/src/test/resources/testdata.avro
new file mode 100644
index 0000000..45308b9
Binary files /dev/null and b/flink-connectors/flink-avro/src/test/resources/testdata.avro differ

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/pom.xml b/flink-connectors/flink-connector-cassandra/pom.xml
new file mode 100644
index 0000000..4ea790a
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/pom.xml
@@ -0,0 +1,179 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-cassandra_2.10</artifactId>
+	<name>flink-connector-cassandra</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<cassandra.version>2.2.5</cassandra.version>
+		<driver.version>3.0.0</driver.version>
+	</properties>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<reuseForks>true</reuseForks>
+					<forkCount>1</forkCount>
+					<argLine>-Xms256m -Xmx2800m -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<version>2.4.1</version>
+				<executions>
+					<!-- Run shade goal on package phase -->
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration combine.self="override">
+							<dependencyReducedPomLocation>${project.basedir}/target/dependency-reduced-pom.xml</dependencyReducedPomLocation>
+							<artifactSet>
+								<includes>
+									<include>com.datastax.cassandra:cassandra-driver-core</include>
+									<include>com.datastax.cassandra:cassandra-driver-mapping</include>
+									<include>com.google.guava:guava</include>
+								</includes>
+							</artifactSet>
+							<relocations>
+								<relocation>
+									<pattern>com.google</pattern>
+									<shadedPattern>org.apache.flink.cassandra.shaded.com.google</shadedPattern>
+									<excludes>
+										<exclude>com.google.protobuf.**</exclude>
+										<exclude>com.google.inject.**</exclude>
+									</excludes>
+								</relocation>
+							</relocations>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+		<dependency>
+			<groupId>com.datastax.cassandra</groupId>
+			<artifactId>cassandra-driver-core</artifactId>
+			<version>${driver.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>log4j-over-slf4j</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>ch.qos.logback</groupId>
+					<artifactId>logback-classic</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+		<dependency>
+			<groupId>com.datastax.cassandra</groupId>
+			<artifactId>cassandra-driver-mapping</artifactId>
+			<version>${driver.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>log4j-over-slf4j</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>ch.qos.logback</groupId>
+					<artifactId>logback-classic</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.cassandra</groupId>
+			<artifactId>cassandra-all</artifactId>
+			<version>${cassandra.version}</version>
+			<scope>test</scope>
+			<exclusions>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>log4j-over-slf4j</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>ch.qos.logback</groupId>
+					<artifactId>logback-classic</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+	</dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java
new file mode 100644
index 0000000..849e023
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.batch.connectors.cassandra;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.google.common.base.Strings;
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
+import org.apache.flink.api.common.io.NonParallelInput;
+import org.apache.flink.api.common.io.RichInputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * InputFormat to read data from Apache Cassandra and generate ${@link Tuple}.
+ *
+ * @param <OUT> type of Tuple
+ */
+public class CassandraInputFormat<OUT extends Tuple> extends RichInputFormat<OUT, InputSplit> implements NonParallelInput {
+	private static final Logger LOG = LoggerFactory.getLogger(CassandraInputFormat.class);
+
+	private final String query;
+	private final ClusterBuilder builder;
+
+	private transient Cluster cluster;
+	private transient Session session;
+	private transient ResultSet resultSet;
+
+	public CassandraInputFormat(String query, ClusterBuilder builder) {
+		Preconditions.checkArgument(!Strings.isNullOrEmpty(query), "Query cannot be null or empty");
+		Preconditions.checkArgument(builder != null, "Builder cannot be null");
+
+		this.query = query;
+		this.builder = builder;
+	}
+
+	@Override
+	public void configure(Configuration parameters) {
+		this.cluster = builder.getCluster();
+	}
+
+	@Override
+	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
+		return cachedStatistics;
+	}
+
+	/**
+	 * Opens a Session and executes the query.
+	 *
+	 * @param ignored
+	 * @throws IOException
+	 */
+	@Override
+	public void open(InputSplit ignored) throws IOException {
+		this.session = cluster.connect();
+		this.resultSet = session.execute(query);
+	}
+
+	@Override
+	public boolean reachedEnd() throws IOException {
+		return resultSet.isExhausted();
+	}
+
+	@Override
+	public OUT nextRecord(OUT reuse) throws IOException {
+		final Row item = resultSet.one();
+		for (int i = 0; i < reuse.getArity(); i++) {
+			reuse.setField(item.getObject(i), i);
+		}
+		return reuse;
+	}
+
+	@Override
+	public InputSplit[] createInputSplits(int minNumSplits) throws IOException {
+		GenericInputSplit[] split = {new GenericInputSplit(0, 1)};
+		return split;
+	}
+
+	@Override
+	public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) {
+		return new DefaultInputSplitAssigner(inputSplits);
+	}
+
+	/**
+	 * Closes all resources used.
+	 */
+	@Override
+	public void close() throws IOException {
+		try {
+			if (session != null) {
+				session.close();
+			}
+		} catch (Exception e) {
+			LOG.error("Error while closing session.", e);
+		}
+
+		try {
+			if (cluster != null ) {
+				cluster.close();
+			}
+		} catch (Exception e) {
+			LOG.error("Error while closing cluster.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java
new file mode 100644
index 0000000..15d8fb3
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormat.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.batch.connectors.cassandra;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.ResultSetFuture;
+import com.datastax.driver.core.Session;
+import com.google.common.base.Strings;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import org.apache.flink.api.common.io.RichOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * OutputFormat to write {@link org.apache.flink.api.java.tuple.Tuple} into Apache Cassandra.
+ *
+ * @param <OUT> type of Tuple
+ */
+public class CassandraOutputFormat<OUT extends Tuple> extends RichOutputFormat<OUT> {
+	private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormat.class);
+
+	private final String insertQuery;
+	private final ClusterBuilder builder;
+
+	private transient Cluster cluster;
+	private transient Session session;
+	private transient PreparedStatement prepared;
+	private transient FutureCallback<ResultSet> callback;
+	private transient Throwable exception = null;
+
+	public CassandraOutputFormat(String insertQuery, ClusterBuilder builder) {
+		Preconditions.checkArgument(!Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+		Preconditions.checkArgument(builder != null, "Builder cannot be null");
+
+		this.insertQuery = insertQuery;
+		this.builder = builder;
+	}
+
+	@Override
+	public void configure(Configuration parameters) {
+		this.cluster = builder.getCluster();
+	}
+
+	/**
+	 * Opens a Session to Cassandra and initializes the prepared statement.
+	 *
+	 * @param taskNumber The number of the parallel instance.
+	 * @throws IOException Thrown, if the output could not be opened due to an
+	 *                     I/O problem.
+	 */
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		this.session = cluster.connect();
+		this.prepared = session.prepare(insertQuery);
+		this.callback = new FutureCallback<ResultSet>() {
+			@Override
+			public void onSuccess(ResultSet ignored) {
+			}
+
+			@Override
+			public void onFailure(Throwable t) {
+				exception = t;
+			}
+		};
+	}
+
+	@Override
+	public void writeRecord(OUT record) throws IOException {
+		if (exception != null) {
+			throw new IOException("write record failed", exception);
+		}
+
+		Object[] fields = new Object[record.getArity()];
+		for (int i = 0; i < record.getArity(); i++) {
+			fields[i] = record.getField(i);
+		}
+		ResultSetFuture result = session.executeAsync(prepared.bind(fields));
+		Futures.addCallback(result, callback);
+	}
+
+	/**
+	 * Closes all resources used.
+	 */
+	@Override
+	public void close() throws IOException {
+		try {
+			if (session != null) {
+				session.close();
+			}
+		} catch (Exception e) {
+			LOG.error("Error while closing session.", e);
+		}
+
+		try {
+			if (cluster != null ) {
+				cluster.close();
+			}
+		} catch (Exception e) {
+			LOG.error("Error while closing cluster.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraCommitter.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraCommitter.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraCommitter.java
new file mode 100644
index 0000000..63b76da
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraCommitter.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cassandra;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * CheckpointCommitter that saves information about completed checkpoints within a separate table in a cassandra
+ * database.
+ * 
+ * <p>Entries are in the form |operator_id | subtask_id | last_completed_checkpoint|
+ */
+public class CassandraCommitter extends CheckpointCommitter {
+
+	private static final long serialVersionUID = 1L;
+	
+	private final ClusterBuilder builder;
+	private transient Cluster cluster;
+	private transient Session session;
+
+	private String keySpace = "flink_auxiliary";
+	private String table = "checkpoints_";
+
+	/**
+	 * A cache of the last committed checkpoint ids per subtask index. This is used to
+	 * avoid redundant round-trips to Cassandra (see {@link #isCheckpointCommitted(int, long)}.
+	 */
+	private final Map<Integer, Long> lastCommittedCheckpoints = new HashMap<>();
+
+	public CassandraCommitter(ClusterBuilder builder) {
+		this.builder = builder;
+		ClosureCleaner.clean(builder, true);
+	}
+
+	public CassandraCommitter(ClusterBuilder builder, String keySpace) {
+		this(builder);
+		this.keySpace = keySpace;
+	}
+
+	/**
+	 * Internally used to set the job ID after instantiation.
+	 */
+	public void setJobId(String id) throws Exception {
+		super.setJobId(id);
+		table += id;
+	}
+
+	/**
+	 * Generates the necessary tables to store information.
+	 *
+	 * @throws Exception
+	 */
+	@Override
+	public void createResource() throws Exception {
+		cluster = builder.getCluster();
+		session = cluster.connect();
+
+		session.execute(String.format("CREATE KEYSPACE IF NOT EXISTS %s with replication={'class':'SimpleStrategy', 'replication_factor':1};", keySpace));
+		session.execute(String.format("CREATE TABLE IF NOT EXISTS %s.%s (sink_id text, sub_id int, checkpoint_id bigint, PRIMARY KEY (sink_id, sub_id));", keySpace, table));
+
+		try {
+			session.close();
+		} catch (Exception e) {
+			LOG.error("Error while closing session.", e);
+		}
+		try {
+			cluster.close();
+		} catch (Exception e) {
+			LOG.error("Error while closing cluster.", e);
+		}
+	}
+
+	@Override
+	public void open() throws Exception {
+		if (builder == null) {
+			throw new RuntimeException("No ClusterBuilder was set.");
+		}
+		cluster = builder.getCluster();
+		session = cluster.connect();
+	}
+
+	@Override
+	public void close() throws Exception {
+		this.lastCommittedCheckpoints.clear();
+		try {
+			session.close();
+		} catch (Exception e) {
+			LOG.error("Error while closing session.", e);
+		}
+		try {
+			cluster.close();
+		} catch (Exception e) {
+			LOG.error("Error while closing cluster.", e);
+		}
+	}
+
+	@Override
+	public void commitCheckpoint(int subtaskIdx, long checkpointId) {
+		String statement = String.format(
+			"UPDATE %s.%s set checkpoint_id=%d where sink_id='%s' and sub_id=%d;",
+			keySpace, table, checkpointId, operatorId, subtaskIdx);
+
+		session.execute(statement);
+		lastCommittedCheckpoints.put(subtaskIdx, checkpointId);
+	}
+
+	@Override
+	public boolean isCheckpointCommitted(int subtaskIdx, long checkpointId) {
+		// Pending checkpointed buffers are committed in ascending order of their
+		// checkpoint id. This way we can tell if a checkpointed buffer was committed
+		// just by asking the third-party storage system for the last checkpoint id
+		// committed by the specified subtask.
+
+		Long lastCommittedCheckpoint = lastCommittedCheckpoints.get(subtaskIdx);
+		if (lastCommittedCheckpoint == null) {
+			String statement = String.format(
+				"SELECT checkpoint_id FROM %s.%s where sink_id='%s' and sub_id=%d;",
+				keySpace, table, operatorId, subtaskIdx);
+
+			Iterator<Row> resultIt = session.execute(statement).iterator();
+			if (resultIt.hasNext()) {
+				lastCommittedCheckpoint = resultIt.next().getLong("checkpoint_id");
+				lastCommittedCheckpoints.put(subtaskIdx, lastCommittedCheckpoint);
+			}
+		}
+		return lastCommittedCheckpoint != null && checkpointId <= lastCommittedCheckpoint;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraPojoSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraPojoSink.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraPojoSink.java
new file mode 100644
index 0000000..650c481
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraPojoSink.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.cassandra;
+
+import com.datastax.driver.mapping.Mapper;
+import com.datastax.driver.mapping.MappingManager;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * Flink Sink to save data into a Cassandra cluster using 
+ * <a href="http://docs.datastax.com/en/drivers/java/2.1/com/datastax/driver/mapping/Mapper.html">Mapper</a>,
+ * which it uses annotations from
+ * <a href="http://docs.datastax.com/en/drivers/java/2.1/com/datastax/driver/mapping/annotations/package-summary.html">
+ * com.datastax.driver.mapping.annotations</a>.
+ *
+ * @param <IN> Type of the elements emitted by this sink
+ */
+public class CassandraPojoSink<IN> extends CassandraSinkBase<IN, Void> {
+
+	private static final long serialVersionUID = 1L;
+
+	protected final Class<IN> clazz;
+	protected transient Mapper<IN> mapper;
+	protected transient MappingManager mappingManager;
+
+	/**
+	 * The main constructor for creating CassandraPojoSink
+	 *
+	 * @param clazz Class<IN> instance
+	 */
+	public CassandraPojoSink(Class<IN> clazz, ClusterBuilder builder) {
+		super(builder);
+		this.clazz = clazz;
+	}
+
+	@Override
+	public void open(Configuration configuration) {
+		super.open(configuration);
+		try {
+			this.mappingManager = new MappingManager(session);
+			this.mapper = mappingManager.mapper(clazz);
+		} catch (Exception e) {
+			throw new RuntimeException("Cannot create CassandraPojoSink with input: " + clazz.getSimpleName(), e);
+		}
+	}
+
+	@Override
+	public ListenableFuture<Void> send(IN value) {
+		return mapper.saveAsync(value);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSink.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSink.java
new file mode 100644
index 0000000..180b638
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSink.java
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cassandra;
+
+import com.datastax.driver.core.Cluster;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.transformations.SinkTransformation;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
+
+/**
+ * This class wraps different Cassandra sink implementations to provide a common interface for all of them.
+ *
+ * @param <IN> input type
+ */
+public class CassandraSink<IN> {
+	private final boolean useDataStreamSink;
+	private DataStreamSink<IN> sink1;
+	private SingleOutputStreamOperator<IN> sink2;
+
+	private CassandraSink(DataStreamSink<IN> sink) {
+		sink1 = sink;
+		useDataStreamSink = true;
+	}
+
+	private CassandraSink(SingleOutputStreamOperator<IN> sink) {
+		sink2 = sink;
+		useDataStreamSink = false;
+	}
+
+	private SinkTransformation<IN> getSinkTransformation() {
+		return sink1.getTransformation();
+	}
+
+	private StreamTransformation<IN> getStreamTransformation() {
+		return sink2.getTransformation();
+	}
+
+	/**
+	 * Sets the name of this sink. This name is
+	 * used by the visualization and logging during runtime.
+	 *
+	 * @return The named sink.
+	 */
+	public CassandraSink<IN> name(String name) {
+		if (useDataStreamSink) {
+			getSinkTransformation().setName(name);
+		} else {
+			getStreamTransformation().setName(name);
+		}
+		return this;
+	}
+
+	/**
+	 * Sets an ID for this operator.
+	 * <p/>
+	 * <p>The specified ID is used to assign the same operator ID across job
+	 * submissions (for example when starting a job from a savepoint).
+	 * <p/>
+	 * <p><strong>Important</strong>: this ID needs to be unique per
+	 * transformation and job. Otherwise, job submission will fail.
+	 *
+	 * @param uid The unique user-specified ID of this transformation.
+	 * @return The operator with the specified ID.
+	 */
+	public CassandraSink<IN> uid(String uid) {
+		if (useDataStreamSink) {
+			getSinkTransformation().setUid(uid);
+		} else {
+			getStreamTransformation().setUid(uid);
+		}
+		return this;
+	}
+
+	/**
+	 * Sets the parallelism for this sink. The degree must be higher than zero.
+	 *
+	 * @param parallelism The parallelism for this sink.
+	 * @return The sink with set parallelism.
+	 */
+	public CassandraSink<IN> setParallelism(int parallelism) {
+		if (useDataStreamSink) {
+			getSinkTransformation().setParallelism(parallelism);
+		} else {
+			getStreamTransformation().setParallelism(parallelism);
+		}
+		return this;
+	}
+
+	/**
+	 * Turns off chaining for this operator so thread co-location will not be
+	 * used as an optimization.
+	 * <p/>
+	 * <p/>
+	 * Chaining can be turned off for the whole
+	 * job by {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#disableOperatorChaining()}
+	 * however it is not advised for performance considerations.
+	 *
+	 * @return The sink with chaining disabled
+	 */
+	public CassandraSink<IN> disableChaining() {
+		if (useDataStreamSink) {
+			getSinkTransformation().setChainingStrategy(ChainingStrategy.NEVER);
+		} else {
+			getStreamTransformation().setChainingStrategy(ChainingStrategy.NEVER);
+		}
+		return this;
+	}
+
+	/**
+	 * Sets the slot sharing group of this operation. Parallel instances of
+	 * operations that are in the same slot sharing group will be co-located in the same
+	 * TaskManager slot, if possible.
+	 * <p/>
+	 * <p>Operations inherit the slot sharing group of input operations if all input operations
+	 * are in the same slot sharing group and no slot sharing group was explicitly specified.
+	 * <p/>
+	 * <p>Initially an operation is in the default slot sharing group. An operation can be put into
+	 * the default group explicitly by setting the slot sharing group to {@code "default"}.
+	 *
+	 * @param slotSharingGroup The slot sharing group name.
+	 */
+	public CassandraSink<IN> slotSharingGroup(String slotSharingGroup) {
+		if (useDataStreamSink) {
+			getSinkTransformation().setSlotSharingGroup(slotSharingGroup);
+		} else {
+			getStreamTransformation().setSlotSharingGroup(slotSharingGroup);
+		}
+		return this;
+	}
+
+	/**
+	 * Writes a DataStream into a Cassandra database.
+	 *
+	 * @param input input DataStream
+	 * @param <IN>  input type
+	 * @return CassandraSinkBuilder, to further configure the sink
+	 */
+	public static <IN, T extends Tuple> CassandraSinkBuilder<IN> addSink(DataStream<IN> input) {
+		if (input.getType() instanceof TupleTypeInfo) {
+			DataStream<T> tupleInput = (DataStream<T>) input;
+			return (CassandraSinkBuilder<IN>) new CassandraTupleSinkBuilder<>(tupleInput, tupleInput.getType(), tupleInput.getType().createSerializer(tupleInput.getExecutionEnvironment().getConfig()));
+		} else {
+			return new CassandraPojoSinkBuilder<>(input, input.getType(), input.getType().createSerializer(input.getExecutionEnvironment().getConfig()));
+		}
+	}
+
+	public abstract static class CassandraSinkBuilder<IN> {
+		protected final DataStream<IN> input;
+		protected final TypeSerializer<IN> serializer;
+		protected final TypeInformation<IN> typeInfo;
+		protected ClusterBuilder builder;
+		protected String query;
+		protected CheckpointCommitter committer;
+		protected boolean isWriteAheadLogEnabled;
+
+		public CassandraSinkBuilder(DataStream<IN> input, TypeInformation<IN> typeInfo, TypeSerializer<IN> serializer) {
+			this.input = input;
+			this.typeInfo = typeInfo;
+			this.serializer = serializer;
+		}
+
+		/**
+		 * Sets the query that is to be executed for every record.
+		 *
+		 * @param query query to use
+		 * @return this builder
+		 */
+		public CassandraSinkBuilder<IN> setQuery(String query) {
+			this.query = query;
+			return this;
+		}
+
+		/**
+		 * Sets the cassandra host to connect to.
+		 *
+		 * @param host host to connect to
+		 * @return this builder
+		 */
+		public CassandraSinkBuilder<IN> setHost(String host) {
+			return setHost(host, 9042);
+		}
+
+		/**
+		 * Sets the cassandra host/port to connect to.
+		 *
+		 * @param host host to connect to
+		 * @param port port to connect to
+		 * @return this builder
+		 */
+		public CassandraSinkBuilder<IN> setHost(final String host, final int port) {
+			if (this.builder != null) {
+				throw new IllegalArgumentException("Builder was already set. You must use either setHost() or setClusterBuilder().");
+			}
+			this.builder = new ClusterBuilder() {
+				@Override
+				protected Cluster buildCluster(Cluster.Builder builder) {
+					return builder.addContactPoint(host).withPort(port).build();
+				}
+			};
+			return this;
+		}
+
+		/**
+		 * Sets the ClusterBuilder for this sink. A ClusterBuilder is used to configure the connection to cassandra.
+		 *
+		 * @param builder ClusterBuilder to configure the connection to cassandra
+		 * @return this builder
+		 */
+		public CassandraSinkBuilder<IN> setClusterBuilder(ClusterBuilder builder) {
+			if (this.builder != null) {
+				throw new IllegalArgumentException("Builder was already set. You must use either setHost() or setClusterBuilder().");
+			}
+			this.builder = builder;
+			return this;
+		}
+
+		/**
+		 * Enables the write-ahead log, which allows exactly-once processing for non-deterministic algorithms that use
+		 * idempotent updates.
+		 *
+		 * @return this builder
+		 */
+		public CassandraSinkBuilder<IN> enableWriteAheadLog() {
+			this.isWriteAheadLogEnabled = true;
+			return this;
+		}
+
+		/**
+		 * Enables the write-ahead log, which allows exactly-once processing for non-deterministic algorithms that use
+		 * idempotent updates.
+		 *
+		 * @param committer CheckpointCommitter, that stores informationa bout completed checkpoints in an external
+		 *                  resource. By default this information is stored within a separate table within Cassandra.
+		 * @return this builder
+		 */
+		public CassandraSinkBuilder<IN> enableWriteAheadLog(CheckpointCommitter committer) {
+			this.isWriteAheadLogEnabled = true;
+			this.committer = committer;
+			return this;
+		}
+
+		/**
+		 * Finalizes the configuration of this sink.
+		 *
+		 * @return finalized sink
+		 * @throws Exception
+		 */
+		public abstract CassandraSink<IN> build() throws Exception;
+
+		protected void sanityCheck() {
+			if (builder == null) {
+				throw new IllegalArgumentException("Cassandra host information must be supplied using either setHost() or setClusterBuilder().");
+			}
+		}
+	}
+
+	public static class CassandraTupleSinkBuilder<IN extends Tuple> extends CassandraSinkBuilder<IN> {
+		public CassandraTupleSinkBuilder(DataStream<IN> input, TypeInformation<IN> typeInfo, TypeSerializer<IN> serializer) {
+			super(input, typeInfo, serializer);
+		}
+
+		@Override
+		protected void sanityCheck() {
+			super.sanityCheck();
+			if (query == null || query.length() == 0) {
+				throw new IllegalArgumentException("Query must not be null or empty.");
+			}
+		}
+
+		@Override
+		public CassandraSink<IN> build() throws Exception {
+			sanityCheck();
+			if (isWriteAheadLogEnabled) {
+				return committer == null
+					? new CassandraSink<>(input.transform("Cassandra Sink", null, new CassandraTupleWriteAheadSink<>(query, serializer, builder, new CassandraCommitter(builder))))
+					: new CassandraSink<>(input.transform("Cassandra Sink", null, new CassandraTupleWriteAheadSink<>(query, serializer, builder, committer)));
+			} else {
+				return new CassandraSink<>(input.addSink(new CassandraTupleSink<IN>(query, builder)).name("Cassandra Sink"));
+			}
+		}
+	}
+
+	public static class CassandraPojoSinkBuilder<IN> extends CassandraSinkBuilder<IN> {
+		public CassandraPojoSinkBuilder(DataStream<IN> input, TypeInformation<IN> typeInfo, TypeSerializer<IN> serializer) {
+			super(input, typeInfo, serializer);
+		}
+
+		@Override
+		protected void sanityCheck() {
+			super.sanityCheck();
+			if (query != null) {
+				throw new IllegalArgumentException("Specifying a query is not allowed when using a Pojo-Stream as input.");
+			}
+		}
+
+		@Override
+		public CassandraSink<IN> build() throws Exception {
+			sanityCheck();
+			if (isWriteAheadLogEnabled) {
+				throw new IllegalArgumentException("Exactly-once guarantees can only be provided for tuple types.");
+			} else {
+				return new CassandraSink<>(input.addSink(new CassandraPojoSink<>(typeInfo.getTypeClass(), builder)).name("Cassandra Sink"));
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java
new file mode 100644
index 0000000..49b1efa
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cassandra;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * CassandraSinkBase is the common abstract class of {@link CassandraPojoSink} and {@link CassandraTupleSink}.
+ *
+ * @param <IN> Type of the elements emitted by this sink
+ */
+public abstract class CassandraSinkBase<IN, V> extends RichSinkFunction<IN> {
+	protected static final Logger LOG = LoggerFactory.getLogger(CassandraSinkBase.class);
+	protected transient Cluster cluster;
+	protected transient Session session;
+
+	protected transient Throwable exception = null;
+	protected transient FutureCallback<V> callback;
+
+	private final ClusterBuilder builder;
+
+	protected CassandraSinkBase(ClusterBuilder builder) {
+		this.builder = builder;
+		ClosureCleaner.clean(builder, true);
+	}
+
+	@Override
+	public void open(Configuration configuration) {
+		this.callback = new FutureCallback<V>() {
+			@Override
+			public void onSuccess(V ignored) {
+			}
+
+			@Override
+			public void onFailure(Throwable t) {
+				exception = t;
+				LOG.error("Error while sending value.", t);
+			}
+		};
+		this.cluster = builder.getCluster();
+		this.session = cluster.connect();
+	}
+
+	@Override
+	public void invoke(IN value) throws Exception {
+		if (exception != null) {
+			throw new IOException("invoke() failed", exception);
+		}
+		ListenableFuture<V> result = send(value);
+		Futures.addCallback(result, callback);
+	}
+
+	public abstract ListenableFuture<V> send(IN value);
+
+	@Override
+	public void close() {
+		try {
+			if (session != null) {
+				session.close();
+			}
+		} catch (Exception e) {
+			LOG.error("Error while closing session.", e);
+		}
+		try {
+			if (cluster != null) {
+				cluster.close();
+			}
+		} catch (Exception e) {
+			LOG.error("Error while closing cluster.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleSink.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleSink.java
new file mode 100644
index 0000000..0a9ef06
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleSink.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.cassandra;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * Flink Sink to save data into a Cassandra cluster.
+ *
+ * @param <IN> Type of the elements emitted by this sink, it must extend {@link Tuple}
+ */
+public class CassandraTupleSink<IN extends Tuple> extends CassandraSinkBase<IN, ResultSet> {
+	private final String insertQuery;
+	private transient PreparedStatement ps;
+
+	public CassandraTupleSink(String insertQuery, ClusterBuilder builder) {
+		super(builder);
+		this.insertQuery = insertQuery;
+	}
+
+	@Override
+	public void open(Configuration configuration) {
+		super.open(configuration);
+		this.ps = session.prepare(insertQuery);
+	}
+
+	@Override
+	public ListenableFuture<ResultSet> send(IN value) {
+		Object[] fields = extract(value);
+		return session.executeAsync(ps.bind(fields));
+	}
+
+	private Object[] extract(IN record) {
+		Object[] al = new Object[record.getArity()];
+		for (int i = 0; i < record.getArity(); i++) {
+			al[i] = record.getField(i);
+		}
+		return al;
+	}
+}


[37/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java
new file mode 100644
index 0000000..42b9682
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+
+import org.junit.Test;
+
+
+@SuppressWarnings("serial")
+public class Kafka010ProducerITCase extends KafkaProducerTestBase {
+
+	@Test
+	public void testCustomPartitioning() {
+		runCustomPartitioningTest();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
new file mode 100644
index 0000000..f15fd45
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
@@ -0,0 +1,420 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import kafka.admin.AdminUtils;
+import kafka.common.KafkaException;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+import kafka.utils.SystemTime$;
+import kafka.utils.ZkUtils;
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.util.NetUtils;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.net.BindException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * An implementation of the KafkaServerProvider for Kafka 0.10
+ */
+public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class);
+	private File tmpZkDir;
+	private File tmpKafkaParent;
+	private List<File> tmpKafkaDirs;
+	private List<KafkaServer> brokers;
+	private TestingServer zookeeper;
+	private String zookeeperConnectionString;
+	private String brokerConnectionString = "";
+	private Properties standardProps;
+	private Properties additionalServerProperties;
+	private boolean secureMode = false;
+	// 6 seconds is default. Seems to be too small for travis. 30 seconds
+	private int zkTimeout = 30000;
+
+	public String getBrokerConnectionString() {
+		return brokerConnectionString;
+	}
+
+	@Override
+	public Properties getStandardProperties() {
+		return standardProps;
+	}
+
+	@Override
+	public Properties getSecureProperties() {
+		Properties prop = new Properties();
+		if(secureMode) {
+			prop.put("security.inter.broker.protocol", "SASL_PLAINTEXT");
+			prop.put("security.protocol", "SASL_PLAINTEXT");
+			prop.put("sasl.kerberos.service.name", "kafka");
+
+			//add special timeout for Travis
+			prop.setProperty("zookeeper.session.timeout.ms", String.valueOf(zkTimeout));
+			prop.setProperty("zookeeper.connection.timeout.ms", String.valueOf(zkTimeout));
+			prop.setProperty("metadata.fetch.timeout.ms","120000");
+		}
+		return prop;
+	}
+
+	@Override
+	public String getVersion() {
+		return "0.10";
+	}
+
+	@Override
+	public List<KafkaServer> getBrokers() {
+		return brokers;
+	}
+
+	@Override
+	public <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, KeyedDeserializationSchema<T> readSchema, Properties props) {
+		return new FlinkKafkaConsumer010<>(topics, readSchema, props);
+	}
+
+	@Override
+	public <T> StreamSink<T> getProducerSink(String topic, KeyedSerializationSchema<T> serSchema, Properties props, KafkaPartitioner<T> partitioner) {
+		FlinkKafkaProducer010<T> prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner);
+		prod.setFlushOnCheckpoint(true);
+		return new StreamSink<>(prod);
+	}
+
+
+	@Override
+	public <T> DataStreamSink<T> produceIntoKafka(DataStream<T> stream, String topic, KeyedSerializationSchema<T> serSchema, Properties props, KafkaPartitioner<T> partitioner) {
+		FlinkKafkaProducer010<T> prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner);
+		prod.setFlushOnCheckpoint(true);
+		return stream.addSink(prod);
+	}
+
+	@Override
+	public KafkaOffsetHandler createOffsetHandler(Properties props) {
+		return new KafkaOffsetHandlerImpl(props);
+	}
+
+	@Override
+	public void restartBroker(int leaderId) throws Exception {
+		brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId)));
+	}
+
+	@Override
+	public int getLeaderToShutDown(String topic) throws Exception {
+		ZkUtils zkUtils = getZkUtils();
+		try {
+			MetadataResponse.PartitionMetadata firstPart = null;
+			do {
+				if (firstPart != null) {
+					LOG.info("Unable to find leader. error code {}", firstPart.error().code());
+					// not the first try. Sleep a bit
+					Thread.sleep(150);
+				}
+
+				List<MetadataResponse.PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkUtils).partitionMetadata();
+				firstPart = partitionMetadata.get(0);
+			}
+			while (firstPart.error().code() != 0);
+
+			return firstPart.leader().id();
+		} finally {
+			zkUtils.close();
+		}
+	}
+
+	@Override
+	public int getBrokerId(KafkaServer server) {
+		return server.config().brokerId();
+	}
+
+	@Override
+	public boolean isSecureRunSupported() {
+		return true;
+	}
+
+	@Override
+	public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) {
+		//increase the timeout since in Travis ZK connection takes long time for secure connection.
+		if(secureMode) {
+			//run only one kafka server to avoid multiple ZK connections from many instances - Travis timeout
+			numKafkaServers = 1;
+			zkTimeout = zkTimeout * 15;
+		}
+
+		this.additionalServerProperties = additionalServerProperties;
+		this.secureMode = secureMode;
+		File tempDir = new File(System.getProperty("java.io.tmpdir"));
+
+		tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
+		assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs());
+
+		tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString()));
+		assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs());
+
+		tmpKafkaDirs = new ArrayList<>(numKafkaServers);
+		for (int i = 0; i < numKafkaServers; i++) {
+			File tmpDir = new File(tmpKafkaParent, "server-" + i);
+			assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
+			tmpKafkaDirs.add(tmpDir);
+		}
+
+		zookeeper = null;
+		brokers = null;
+
+		try {
+			zookeeper = new TestingServer(-	1, tmpZkDir);
+			zookeeperConnectionString = zookeeper.getConnectString();
+			LOG.info("Starting Zookeeper with zookeeperConnectionString: {}", zookeeperConnectionString);
+
+			LOG.info("Starting KafkaServer");
+			brokers = new ArrayList<>(numKafkaServers);
+
+			for (int i = 0; i < numKafkaServers; i++) {
+				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i)));
+
+				if(secureMode) {
+					brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.SASL_PLAINTEXT)) + ",";
+				} else {
+					brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ",";
+				}
+			}
+
+			LOG.info("ZK and KafkaServer started.");
+		}
+		catch (Throwable t) {
+			t.printStackTrace();
+			fail("Test setup failed: " + t.getMessage());
+		}
+
+		standardProps = new Properties();
+		standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
+		standardProps.setProperty("bootstrap.servers", brokerConnectionString);
+		standardProps.setProperty("group.id", "flink-tests");
+		standardProps.setProperty("enable.auto.commit", "false");
+		standardProps.setProperty("zookeeper.session.timeout.ms", String.valueOf(zkTimeout));
+		standardProps.setProperty("zookeeper.connection.timeout.ms", String.valueOf(zkTimeout));
+		standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning. (earliest is kafka 0.10 value)
+		standardProps.setProperty("max.partition.fetch.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
+	}
+
+	@Override
+	public void shutdown() {
+		for (KafkaServer broker : brokers) {
+			if (broker != null) {
+				broker.shutdown();
+			}
+		}
+		brokers.clear();
+
+		if (zookeeper != null) {
+			try {
+				zookeeper.stop();
+			}
+			catch (Exception e) {
+				LOG.warn("ZK.stop() failed", e);
+			}
+			zookeeper = null;
+		}
+
+		// clean up the temp spaces
+
+		if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpKafkaParent);
+			}
+			catch (Exception e) {
+				// ignore
+			}
+		}
+		if (tmpZkDir != null && tmpZkDir.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpZkDir);
+			}
+			catch (Exception e) {
+				// ignore
+			}
+		}
+	}
+
+	public ZkUtils getZkUtils() {
+		ZkClient creator = new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
+				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
+		return ZkUtils.apply(creator, false);
+	}
+
+	@Override
+	public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor, Properties topicConfig) {
+		// create topic with one client
+		LOG.info("Creating topic {}", topic);
+
+		ZkUtils zkUtils = getZkUtils();
+		try {
+			AdminUtils.createTopic(zkUtils, topic, numberOfPartitions, replicationFactor, topicConfig, kafka.admin.RackAwareMode.Enforced$.MODULE$);
+		} finally {
+			zkUtils.close();
+		}
+
+		// validate that the topic has been created
+		final long deadline = System.currentTimeMillis() + 30000;
+		do {
+			try {
+				if(secureMode) {
+					//increase wait time since in Travis ZK timeout occurs frequently
+					int wait = zkTimeout / 100;
+					LOG.info("waiting for {} msecs before the topic {} can be checked", wait, topic);
+					Thread.sleep(wait);
+				} else {
+					Thread.sleep(100);
+				}
+			} catch (InterruptedException e) {
+				// restore interrupted state
+			}
+			// we could use AdminUtils.topicExists(zkUtils, topic) here, but it's results are
+			// not always correct.
+
+			// create a new ZK utils connection
+			ZkUtils checkZKConn = getZkUtils();
+			if(AdminUtils.topicExists(checkZKConn, topic)) {
+				checkZKConn.close();
+				return;
+			}
+			checkZKConn.close();
+		}
+		while (System.currentTimeMillis() < deadline);
+		fail("Test topic could not be created");
+	}
+
+	@Override
+	public void deleteTestTopic(String topic) {
+		ZkUtils zkUtils = getZkUtils();
+		try {
+			LOG.info("Deleting topic {}", topic);
+
+			ZkClient zk = new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
+				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
+
+			AdminUtils.deleteTopic(zkUtils, topic);
+
+			zk.close();
+		} finally {
+			zkUtils.close();
+		}
+	}
+
+	/**
+	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
+	 */
+	protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception {
+		Properties kafkaProperties = new Properties();
+
+		// properties have to be Strings
+		kafkaProperties.put("advertised.host.name", KAFKA_HOST);
+		kafkaProperties.put("broker.id", Integer.toString(brokerId));
+		kafkaProperties.put("log.dir", tmpFolder.toString());
+		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
+		kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
+		kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
+
+		// for CI stability, increase zookeeper session timeout
+		kafkaProperties.put("zookeeper.session.timeout.ms", zkTimeout);
+		kafkaProperties.put("zookeeper.connection.timeout.ms", zkTimeout);
+		if(additionalServerProperties != null) {
+			kafkaProperties.putAll(additionalServerProperties);
+		}
+
+		final int numTries = 5;
+
+		for (int i = 1; i <= numTries; i++) {
+			int kafkaPort = NetUtils.getAvailablePort();
+			kafkaProperties.put("port", Integer.toString(kafkaPort));
+
+			//to support secure kafka cluster
+			if(secureMode) {
+				LOG.info("Adding Kafka secure configurations");
+				kafkaProperties.put("listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort);
+				kafkaProperties.put("advertised.listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort);
+				kafkaProperties.putAll(getSecureProperties());
+			}
+
+			KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
+
+			try {
+				scala.Option<String> stringNone = scala.Option.apply(null);
+				KafkaServer server = new KafkaServer(kafkaConfig, SystemTime$.MODULE$, stringNone);
+				server.startup();
+				return server;
+			}
+			catch (KafkaException e) {
+				if (e.getCause() instanceof BindException) {
+					// port conflict, retry...
+					LOG.info("Port conflict when starting Kafka Broker. Retrying...");
+				}
+				else {
+					throw e;
+				}
+			}
+		}
+
+		throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
+	}
+
+	private class KafkaOffsetHandlerImpl implements KafkaOffsetHandler {
+
+		private final KafkaConsumer<byte[], byte[]> offsetClient;
+
+		public KafkaOffsetHandlerImpl(Properties props) {
+			offsetClient = new KafkaConsumer<>(props);
+		}
+
+		@Override
+		public Long getCommittedOffset(String topicName, int partition) {
+			OffsetAndMetadata committed = offsetClient.committed(new TopicPartition(topicName, partition));
+			return (committed != null) ? committed.offset() : null;
+		}
+
+		@Override
+		public void close() {
+			offsetClient.close();
+		}
+	}
+
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/pom.xml b/flink-connectors/flink-connector-kafka-0.8/pom.xml
new file mode 100644
index 0000000..d1fecb6
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/pom.xml
@@ -0,0 +1,219 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-kafka-0.8_2.10</artifactId>
+	<name>flink-connector-kafka-0.8</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<kafka.version>0.8.2.2</kafka.version>
+	</properties>
+
+	<dependencies>
+
+		<!-- core dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-curator-recipes</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project,
+			won't depend on flink-table. -->
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.kafka</groupId>
+			<artifactId>kafka_${scala.binary.version}</artifactId>
+			<version>${kafka.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>com.sun.jmx</groupId>
+					<artifactId>jmxri</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.sun.jdmk</groupId>
+					<artifactId>jmxtools</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>log4j</groupId>
+					<artifactId>log4j</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>slf4j-simple</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>net.sf.jopt-simple</groupId>
+					<artifactId>jopt-simple</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.scala-lang</groupId>
+					<artifactId>scala-reflect</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.scala-lang</groupId>
+					<artifactId>scala-compiler</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.yammer.metrics</groupId>
+					<artifactId>metrics-annotation</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.xerial.snappy</groupId>
+					<artifactId>snappy-java</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-test</artifactId>
+			<version>${curator.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-metrics-jmx</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+	</dependencies>
+
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
+					<forkCount>1</forkCount>
+				</configuration>
+			</plugin>
+			<!-- Relocate curator -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet>
+								<includes combine.children="append">
+									<include>org.apache.flink:flink-shaded-curator-recipes</include>
+								</includes>
+							</artifactSet>
+							<relocations combine.children="append">
+								<relocation>
+									<pattern>org.apache.curator</pattern>
+									<shadedPattern>org.apache.flink.shaded.org.apache.curator</shadedPattern>
+								</relocation>
+							</relocations>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+	
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
new file mode 100644
index 0000000..0aacccd
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import kafka.api.OffsetRequest;
+import kafka.cluster.Broker;
+import kafka.common.ErrorMapping;
+import kafka.javaapi.PartitionMetadata;
+import kafka.javaapi.TopicMetadata;
+import kafka.javaapi.TopicMetadataRequest;
+import kafka.javaapi.consumer.SimpleConsumer;
+
+import org.apache.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.Kafka08Fetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader;
+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.PropertiesUtil;
+import org.apache.flink.util.SerializedValue;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.Node;
+
+import java.net.InetAddress;
+import java.net.URL;
+import java.net.UnknownHostException;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import static org.apache.flink.util.PropertiesUtil.getInt;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
+ * Apache Kafka 0.8.x. The consumer can run in multiple parallel instances, each of which will pull
+ * data from one or more Kafka partitions. 
+ * 
+ * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
+ * during a failure, and that the computation processes elements "exactly once". 
+ * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
+ * 
+ * <p>Flink's Kafka Consumer is designed to be compatible with Kafka's High-Level Consumer API (0.8.x).
+ * Most of Kafka's configuration variables can be used with this consumer as well:
+ *         <ul>
+ *             <li>socket.timeout.ms</li>
+ *             <li>socket.receive.buffer.bytes</li>
+ *             <li>fetch.message.max.bytes</li>
+ *             <li>auto.offset.reset with the values "largest", "smallest"</li>
+ *             <li>fetch.wait.max.ms</li>
+ *         </ul>
+ *     </li>
+ * </ul>
+ * 
+ * <h1>Offset handling</h1>
+ * 
+ * <p>Offsets whose records have been read and are checkpointed will be committed back to ZooKeeper
+ * by the offset handler. In addition, the offset handler finds the point where the source initially
+ * starts reading from the stream, when the streaming job is started.</p>
+ *
+ * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
+ * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
+ * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
+ * has consumed a topic.</p>
+ *
+ * <p>If checkpointing is disabled, the consumer will periodically commit the current offset
+ * to Zookeeper.</p>
+ *
+ * <p>When using a Kafka topic to send data between Flink jobs, we recommend using the
+ * {@see TypeInformationSerializationSchema} and {@see TypeInformationKeyValueSerializationSchema}.</p>
+ * 
+ * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
+ * is constructed. That means that the client that submits the program needs to be able to
+ * reach the Kafka brokers or ZooKeeper.</p>
+ */
+public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
+
+	private static final long serialVersionUID = -6272159445203409112L;
+
+	/** Configuration key for the number of retries for getting the partition info */
+	public static final String GET_PARTITIONS_RETRIES_KEY = "flink.get-partitions.retry";
+
+	/** Default number of retries for getting the partition info. One retry means going through the full list of brokers */
+	public static final int DEFAULT_GET_PARTITIONS_RETRIES = 3;
+
+	// ------------------------------------------------------------------------
+
+	/** The properties to parametrize the Kafka consumer and ZooKeeper client */ 
+	private final Properties kafkaProperties;
+
+	/** The behavior when encountering an invalid offset (see {@link OffsetRequest}) */
+	private final long invalidOffsetBehavior;
+
+	/** The interval in which to automatically commit (-1 if deactivated) */
+	private final long autoCommitInterval;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.8.x
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
+	public FlinkKafkaConsumer08(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
+		this(Collections.singletonList(topic), valueDeserializer, props);
+	}
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.8.x
+	 *
+	 * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value
+	 * pairs, offsets, and topic names from Kafka.
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param deserializer
+	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
+	public FlinkKafkaConsumer08(String topic, KeyedDeserializationSchema<T> deserializer, Properties props) {
+		this(Collections.singletonList(topic), deserializer, props);
+	}
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.8.x
+	 *
+	 * This constructor allows passing multiple topics to the consumer.
+	 *
+	 * @param topics
+	 *           The Kafka topics to read from.
+	 * @param deserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties that are used to configure both the fetcher and the offset handler.
+	 */
+	public FlinkKafkaConsumer08(List<String> topics, DeserializationSchema<T> deserializer, Properties props) {
+		this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props);
+	}
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.8.x
+	 *
+	 * This constructor allows passing multiple topics and a key/value deserialization schema.
+	 * 
+	 * @param topics
+	 *           The Kafka topics to read from.
+	 * @param deserializer
+	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           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(topics, deserializer);
+
+		checkNotNull(topics, "topics");
+		this.kafkaProperties = checkNotNull(props, "props");
+
+		// validate the zookeeper properties
+		validateZooKeeperConfig(props);
+
+		this.invalidOffsetBehavior = getInvalidOffsetBehavior(props);
+		this.autoCommitInterval = PropertiesUtil.getLong(props, "auto.commit.interval.ms", 60000);
+	}
+
+	@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(kafkaProperties.getProperty(KEY_DISABLE_METRICS, "false"));
+
+		return new Kafka08Fetcher<>(sourceContext, thisSubtaskPartitions,
+				watermarksPeriodic, watermarksPunctuated,
+				runtimeContext, deserializer, kafkaProperties,
+				invalidOffsetBehavior, autoCommitInterval, useMetrics);
+	}
+
+	@Override
+	protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
+		// Connect to a broker to get the partitions for all topics
+		List<KafkaTopicPartition> partitionInfos =
+			KafkaTopicPartition.dropLeaderData(getPartitionsForTopic(topics, kafkaProperties));
+
+		if (partitionInfos.size() == 0) {
+			throw new RuntimeException(
+				"Unable to retrieve any partitions for the requested topics " + topics +
+					". Please check previous log entries");
+		}
+
+		if (LOG.isInfoEnabled()) {
+			logPartitionInfo(LOG, partitionInfos);
+		}
+
+		return partitionInfos;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Kafka / ZooKeeper communication utilities
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Send request to Kafka to get partitions for topic.
+	 * 
+	 * @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(List<String> topics, Properties properties) {
+		String seedBrokersConfString = properties.getProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
+		final int numRetries = getInt(properties, GET_PARTITIONS_RETRIES_KEY, DEFAULT_GET_PARTITIONS_RETRIES);
+		
+		checkNotNull(seedBrokersConfString, "Configuration property %s not set", ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
+		String[] seedBrokers = seedBrokersConfString.split(",");
+		List<KafkaTopicPartitionLeader> partitions = new ArrayList<>();
+
+		final String clientId = "flink-kafka-consumer-partition-lookup";
+		final int soTimeout = getInt(properties, "socket.timeout.ms", 30000);
+		final int bufferSize = getInt(properties, "socket.receive.buffer.bytes", 65536);
+
+		Random rnd = new Random();
+		retryLoop: for (int retry = 0; retry < numRetries; retry++) {
+			// we pick a seed broker randomly to avoid overloading the first broker with all the requests when the
+			// parallel source instances start. Still, we try all available brokers.
+			int index = rnd.nextInt(seedBrokers.length);
+			brokersLoop: for (int arrIdx = 0; arrIdx < seedBrokers.length; arrIdx++) {
+				String seedBroker = seedBrokers[index];
+				LOG.info("Trying to get topic metadata from broker {} in try {}/{}", seedBroker, retry, numRetries);
+				if (++index == seedBrokers.length) {
+					index = 0;
+				}
+
+				URL brokerUrl = NetUtils.getCorrectHostnamePort(seedBroker);
+				SimpleConsumer consumer = null;
+				try {
+					consumer = new SimpleConsumer(brokerUrl.getHost(), brokerUrl.getPort(), soTimeout, bufferSize, clientId);
+
+					TopicMetadataRequest req = new TopicMetadataRequest(topics);
+					kafka.javaapi.TopicMetadataResponse resp = consumer.send(req);
+
+					List<TopicMetadata> metaData = resp.topicsMetadata();
+
+					// clear in case we have an incomplete list from previous tries
+					partitions.clear();
+					for (TopicMetadata item : metaData) {
+						if (item.errorCode() != ErrorMapping.NoError()) {
+							// warn and try more brokers
+							LOG.warn("Error while getting metadata from broker " + seedBroker + " to find partitions " +
+									"for " + topics.toString() + ". Error: " + ErrorMapping.exceptionFor(item.errorCode()).getMessage());
+							continue brokersLoop;
+						}
+						if (!topics.contains(item.topic())) {
+							LOG.warn("Received metadata from topic " + item.topic() + " even though it was not requested. Skipping ...");
+							continue brokersLoop;
+						}
+						for (PartitionMetadata part : item.partitionsMetadata()) {
+							Node leader = brokerToNode(part.leader());
+							KafkaTopicPartition ktp = new KafkaTopicPartition(item.topic(), part.partitionId());
+							KafkaTopicPartitionLeader pInfo = new KafkaTopicPartitionLeader(ktp, leader);
+							partitions.add(pInfo);
+						}
+					}
+					break retryLoop; // leave the loop through the brokers
+				} catch (Exception e) {
+					//validates seed brokers in case of a ClosedChannelException
+					validateSeedBrokers(seedBrokers, e);
+					LOG.warn("Error communicating with broker " + seedBroker + " to find partitions for " + topics.toString() + "." +
+							"" + e.getClass() + ". Message: " + e.getMessage());
+					LOG.debug("Detailed trace", e);
+					// we sleep a bit. Retrying immediately doesn't make sense in cases where Kafka is reorganizing the leader metadata
+					try {
+						Thread.sleep(500);
+					} catch (InterruptedException e1) {
+						// sleep shorter.
+					}
+				} finally {
+					if (consumer != null) {
+						consumer.close();
+					}
+				}
+			} // brokers loop
+		} // retries loop
+		return partitions;
+	}
+
+	/**
+	 * Turn a broker instance into a node instance
+	 * @param broker broker instance
+	 * @return Node representing the given broker
+	 */
+	private static Node brokerToNode(Broker broker) {
+		return new Node(broker.id(), broker.host(), broker.port());
+	}
+
+	/**
+	 * Validate the ZK configuration, checking for required parameters
+	 * @param props Properties to check
+	 */
+	protected static void validateZooKeeperConfig(Properties props) {
+		if (props.getProperty("zookeeper.connect") == null) {
+			throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set in the properties");
+		}
+		if (props.getProperty(ConsumerConfig.GROUP_ID_CONFIG) == null) {
+			throw new IllegalArgumentException("Required property '" + ConsumerConfig.GROUP_ID_CONFIG
+					+ "' has not been set in the properties");
+		}
+		
+		try {
+			//noinspection ResultOfMethodCallIgnored
+			Integer.parseInt(props.getProperty("zookeeper.session.timeout.ms", "0"));
+		}
+		catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Property 'zookeeper.session.timeout.ms' is not a valid integer");
+		}
+		
+		try {
+			//noinspection ResultOfMethodCallIgnored
+			Integer.parseInt(props.getProperty("zookeeper.connection.timeout.ms", "0"));
+		}
+		catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Property 'zookeeper.connection.timeout.ms' is not a valid integer");
+		}
+	}
+
+	/**
+	 * Validate that at least one seed broker is valid in case of a
+	 * ClosedChannelException.
+	 * 
+	 * @param seedBrokers
+	 *            array containing the seed brokers e.g. ["host1:port1",
+	 *            "host2:port2"]
+	 * @param exception
+	 *            instance
+	 */
+	private static void validateSeedBrokers(String[] seedBrokers, Exception exception) {
+		if (!(exception instanceof ClosedChannelException)) {
+			return;
+		}
+		int unknownHosts = 0;
+		for (String broker : seedBrokers) {
+			URL brokerUrl = NetUtils.getCorrectHostnamePort(broker.trim());
+			try {
+				InetAddress.getByName(brokerUrl.getHost());
+			} catch (UnknownHostException e) {
+				unknownHosts++;
+			}
+		}
+		// throw meaningful exception if all the provided hosts are invalid
+		if (unknownHosts == seedBrokers.length) {
+			throw new IllegalArgumentException("All the servers provided in: '"
+					+ ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + "' config are invalid. (unknown hosts)");
+		}
+	}
+
+	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/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
new file mode 100644
index 0000000..56ccd0b
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * THIS CLASS IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
+ */
+@Deprecated
+public class FlinkKafkaConsumer081<T> extends FlinkKafkaConsumer08<T> {
+
+	private static final long serialVersionUID = -5649906773771949146L;
+
+	/**
+	 * THIS CONSTRUCTOR IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
+	 */
+	@Deprecated
+	public FlinkKafkaConsumer081(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
+		super(topic, valueDeserializer, props);
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
new file mode 100644
index 0000000..0520336
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * THIS CLASS IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
+ */
+@Deprecated
+public class FlinkKafkaConsumer082<T> extends FlinkKafkaConsumer08<T> {
+
+	private static final long serialVersionUID = -5649906773771949146L;
+
+	/**
+	 * THIS CONSTRUCTOR IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
+	 */
+	@Deprecated
+	public FlinkKafkaConsumer082(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
+		super(topic, valueDeserializer, props);
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
new file mode 100644
index 0000000..1c2e0b7
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import java.util.Properties;
+
+
+/**
+ * THIS CLASS IS DEPRECATED. Use FlinkKafkaProducer08 instead.
+ */
+@Deprecated
+public class FlinkKafkaProducer<IN> extends FlinkKafkaProducer08<IN>  {
+
+	@Deprecated
+	public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
+		super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), null);
+	}
+
+	@Deprecated
+	public FlinkKafkaProducer(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
+		super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, null);
+	}
+
+	@Deprecated
+	public FlinkKafkaProducer(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+		super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
+
+	}
+
+	@Deprecated
+	public FlinkKafkaProducer(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
+		super(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), null);
+	}
+
+	@Deprecated
+	public FlinkKafkaProducer(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
+		super(topicId, serializationSchema, producerConfig, null);
+	}
+
+	@Deprecated
+	public FlinkKafkaProducer(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+		super(topicId, serializationSchema, producerConfig, customPartitioner);
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
new file mode 100644
index 0000000..65de5fc
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import java.util.Properties;
+
+
+/**
+ * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.8.
+ *
+ * Please note that this producer does not have any reliability guarantees.
+ *
+ * @param <IN> Type of the messages to write into Kafka.
+ */
+public class FlinkKafkaProducer08<IN> extends FlinkKafkaProducerBase<IN>  {
+
+	private static final long serialVersionUID = 1L;
+
+	// ------------------- Keyless serialization schema constructors ----------------------
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+	 * the topic.
+	 *
+	 * @param brokerList
+	 *			Comma separated addresses of the brokers
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined (keyless) serialization schema.
+	 */
+	public FlinkKafkaProducer08(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
+		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner<IN>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+	 * the topic.
+	 *
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined (keyless) serialization schema.
+	 * @param producerConfig
+	 * 			Properties with the producer configuration.
+	 */
+	public FlinkKafkaProducer08(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
+		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner<IN>());
+	}
+
+	/**
+	 * The main constructor for creating a FlinkKafkaProducer.
+	 *
+	 * @param topicId The topic to write data to
+	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
+	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+	 * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
+	 */
+	public FlinkKafkaProducer08(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
+		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
+
+	}
+
+	// ------------------- Key/Value serialization schema constructors ----------------------
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+	 * the topic.
+	 *
+	 * @param brokerList
+	 *			Comma separated addresses of the brokers
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined serialization schema supporting key/value messages
+	 */
+	public FlinkKafkaProducer08(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
+		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner<IN>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+	 * the topic.
+	 *
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined serialization schema supporting key/value messages
+	 * @param producerConfig
+	 * 			Properties with the producer configuration.
+	 */
+	public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
+		this(topicId, serializationSchema, producerConfig, new FixedPartitioner<IN>());
+	}
+
+	/**
+	 * The main constructor for creating a FlinkKafkaProducer.
+	 *
+	 * @param topicId The topic to write data to
+	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
+	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+	 * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
+	 */
+	public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
+		super(topicId, serializationSchema, producerConfig, customPartitioner);
+	}
+
+	@Override
+	protected void flush() {
+		// The Kafka 0.8 producer doesn't support flushing, we wait here
+		// until all pending records are confirmed
+		synchronized (pendingRecordsLock) {
+			while (pendingRecords > 0) {
+				try {
+					pendingRecordsLock.wait();
+				} catch (InterruptedException e) {
+					// this can be interrupted when the Task has been cancelled.
+					// by throwing an exception, we ensure that this checkpoint doesn't get confirmed
+					throw new RuntimeException("Flushing got interrupted while checkpointing", e);
+				}
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
new file mode 100644
index 0000000..b155576
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Kafka 0.8 {@link KafkaTableSink} that serializes data in JSON format.
+ */
+public class Kafka08JsonTableSink extends KafkaJsonTableSink {
+
+	/**
+	 * Creates {@link KafkaTableSink} for Kafka 0.8
+	 *
+	 * @param topic topic in Kafka to which table is written
+	 * @param properties properties to connect to Kafka
+	 * @param partitioner Kafka partitioner
+	 */
+	public Kafka08JsonTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner) {
+		super(topic, properties, partitioner);
+	}
+
+	@Override
+	protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties, SerializationSchema<Row> serializationSchema, KafkaPartitioner<Row> partitioner) {
+		return new FlinkKafkaProducer08<>(topic, serializationSchema, properties, partitioner);
+	}
+
+	@Override
+	protected Kafka08JsonTableSink createCopy() {
+		return new Kafka08JsonTableSink(topic, properties, partitioner);
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
new file mode 100644
index 0000000..63bb57e
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.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;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Kafka {@link StreamTableSource} for Kafka 0.8.
+ */
+public class Kafka08JsonTableSource extends KafkaJsonTableSource {
+
+	/**
+	 * Creates a Kafka 0.8 JSON {@link StreamTableSource}.
+	 *
+	 * @param topic      Kafka topic to consume.
+	 * @param properties Properties for the Kafka consumer.
+	 * @param fieldNames Row field names.
+	 * @param fieldTypes Row field types.
+	 */
+	public Kafka08JsonTableSource(
+			String topic,
+			Properties properties,
+			String[] fieldNames,
+			TypeInformation<?>[] fieldTypes) {
+
+		super(topic, properties, fieldNames, fieldTypes);
+	}
+
+	/**
+	 * Creates a Kafka 0.8 JSON {@link StreamTableSource}.
+	 *
+	 * @param topic      Kafka topic to consume.
+	 * @param properties Properties for the Kafka consumer.
+	 * @param fieldNames Row field names.
+	 * @param fieldTypes Row field types.
+	 */
+	public Kafka08JsonTableSource(
+			String topic,
+			Properties properties,
+			String[] fieldNames,
+			Class<?>[] fieldTypes) {
+
+		super(topic, properties, fieldNames, fieldTypes);
+	}
+
+	@Override
+	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
+		return new FlinkKafkaConsumer08<>(topic, deserializationSchema, properties);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
new file mode 100644
index 0000000..8f51237
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Kafka {@link StreamTableSource} for Kafka 0.8.
+ */
+public class Kafka08TableSource extends KafkaTableSource {
+
+	/**
+	 * Creates a Kafka 0.8 {@link StreamTableSource}.
+	 *
+	 * @param topic                 Kafka topic to consume.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param deserializationSchema Deserialization schema to use for Kafka records.
+	 * @param fieldNames            Row field names.
+	 * @param fieldTypes            Row field types.
+	 */
+	public Kafka08TableSource(
+			String topic,
+			Properties properties,
+			DeserializationSchema<Row> deserializationSchema,
+			String[] fieldNames,
+			TypeInformation<?>[] fieldTypes) {
+
+		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
+	}
+
+	/**
+	 * Creates a Kafka 0.8 {@link StreamTableSource}.
+	 *
+	 * @param topic                 Kafka topic to consume.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param deserializationSchema Deserialization schema to use for Kafka records.
+	 * @param fieldNames            Row field names.
+	 * @param fieldTypes            Row field types.
+	 */
+	public Kafka08TableSource(
+			String topic,
+			Properties properties,
+			DeserializationSchema<Row> deserializationSchema,
+			String[] fieldNames,
+			Class<?>[] fieldTypes) {
+
+		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
+	}
+
+	@Override
+	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
+		return new FlinkKafkaConsumer08<>(topic, deserializationSchema, properties);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java
new file mode 100644
index 0000000..23ff276
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java
@@ -0,0 +1,507 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A special form of blocking queue with two additions:
+ * <ol>
+ *     <li>The queue can be closed atomically when empty. Adding elements after the queue
+ *         is closed fails. This allows queue consumers to atomically discover that no elements
+ *         are available and mark themselves as shut down.</li>
+ *     <li>The queue allows to poll batches of elements in one polling call.</li>
+ * </ol>
+ * 
+ * The queue has no capacity restriction and is safe for multiple producers and consumers.
+ * 
+ * <p>Note: Null elements are prohibited.
+ * 
+ * @param <E> The type of elements in the queue.
+ */
+public class ClosableBlockingQueue<E> {
+
+	/** The lock used to make queue accesses and open checks atomic */
+	private final ReentrantLock lock;
+	
+	/** The condition on which blocking get-calls wait if the queue is empty */
+	private final Condition nonEmpty;
+	
+	/** The deque of elements */
+	private final ArrayDeque<E> elements;
+	
+	/** Flag marking the status of the queue */
+	private volatile boolean open;
+	
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new empty queue.
+	 */
+	public ClosableBlockingQueue() {
+		this(10);
+	}
+
+	/**
+	 * Creates a new empty queue, reserving space for at least the specified number
+	 * of elements. The queu can still grow, of more elements are added than the
+	 * reserved space.
+	 * 
+	 * @param initialSize The number of elements to reserve space for.
+	 */
+	public ClosableBlockingQueue(int initialSize) {
+		this.lock = new ReentrantLock(true);
+		this.nonEmpty = this.lock.newCondition();
+		
+		this.elements = new ArrayDeque<>(initialSize);
+		this.open = true;
+		
+		
+	}
+
+	/**
+	 * Creates a new queue that contains the given elements.
+	 * 
+	 * @param initialElements The elements to initially add to the queue.
+	 */
+	public ClosableBlockingQueue(Collection<? extends E> initialElements) {
+		this(initialElements.size());
+		this.elements.addAll(initialElements);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Size and status
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the number of elements currently in the queue.
+	 * @return The number of elements currently in the queue.
+	 */
+	public int size() {
+		lock.lock();
+		try {
+			return elements.size();
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Checks whether the queue is empty (has no elements).
+	 * @return True, if the queue is empty; false, if it is non-empty.
+	 */
+	public boolean isEmpty() {
+		return size() == 0;
+	}
+
+	/**
+	 * Checks whether the queue is currently open, meaning elements can be added and polled.
+	 * @return True, if the queue is open; false, if it is closed.
+	 */
+	public boolean isOpen() {
+		return open;
+	}
+	
+	/**
+	 * Tries to close the queue. Closing the queue only succeeds when no elements are
+	 * in the queue when this method is called. Checking whether the queue is empty, and
+	 * marking the queue as closed is one atomic operation.
+	 *
+	 * @return True, if the queue is closed, false if the queue remains open.
+	 */
+	public boolean close() {
+		lock.lock();
+		try {
+			if (open) {
+				if (elements.isEmpty()) {
+					open = false;
+					nonEmpty.signalAll();
+					return true;
+				} else {
+					return false;
+				}
+			}
+			else {
+				// already closed
+				return true;
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Adding / Removing elements
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Tries to add an element to the queue, if the queue is still open. Checking whether the queue
+	 * is open and adding the element is one atomic operation.
+	 * 
+	 * <p>Unlike the {@link #add(Object)} method, this method never throws an exception,
+	 * but only indicates via the return code if the element was added or the
+	 * queue was closed.
+	 * 
+	 * @param element The element to add.
+	 * @return True, if the element was added, false if the queue was closes.
+	 */
+	public boolean addIfOpen(E element) {
+		requireNonNull(element);
+		
+		lock.lock();
+		try {
+			if (open) {
+				elements.addLast(element);
+				if (elements.size() == 1) {
+					nonEmpty.signalAll();
+				}
+			}
+			return open;
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Adds the element to the queue, or fails with an exception, if the queue is closed.
+	 * Checking whether the queue is open and adding the element is one atomic operation.
+	 * 
+	 * @param element The element to add.
+	 * @throws IllegalStateException Thrown, if the queue is closed.
+	 */
+	public void add(E element) throws IllegalStateException {
+		requireNonNull(element);
+
+		lock.lock();
+		try {
+			if (open) {
+				elements.addLast(element);
+				if (elements.size() == 1) {
+					nonEmpty.signalAll();
+				}
+			} else {
+				throw new IllegalStateException("queue is closed");
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Returns the queue's next element without removing it, if the queue is non-empty.
+	 * Otherwise, returns null. 
+	 *
+	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
+	 * Checking whether the queue is open and getting the next element is one atomic operation.
+	 * 
+	 * <p>This method never blocks.
+	 * 
+	 * @return The queue's next element, or null, if the queue is empty.
+	 * @throws IllegalStateException Thrown, if the queue is closed.
+	 */
+	public E peek() {
+		lock.lock();
+		try {
+			if (open) {
+				if (elements.size() > 0) {
+					return elements.getFirst();
+				} else {
+					return null;
+				}
+			} else {
+				throw new IllegalStateException("queue is closed");
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Returns the queue's next element and removes it, the queue is non-empty.
+	 * Otherwise, this method returns null. 
+	 *
+	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
+	 * Checking whether the queue is open and removing the next element is one atomic operation.
+	 *
+	 * <p>This method never blocks.
+	 *
+	 * @return The queue's next element, or null, if the queue is empty.
+	 * @throws IllegalStateException Thrown, if the queue is closed.
+	 */
+	public E poll() {
+		lock.lock();
+		try {
+			if (open) {
+				if (elements.size() > 0) {
+					return elements.removeFirst();
+				} else {
+					return null;
+				}
+			} else {
+				throw new IllegalStateException("queue is closed");
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Returns all of the queue's current elements in a list, if the queue is non-empty.
+	 * Otherwise, this method returns null. 
+	 *
+	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
+	 * Checking whether the queue is open and removing the elements is one atomic operation.
+	 *
+	 * <p>This method never blocks.
+	 *
+	 * @return All of the queue's elements, or null, if the queue is empty.
+	 * @throws IllegalStateException Thrown, if the queue is closed.
+	 */
+	public List<E> pollBatch() {
+		lock.lock();
+		try {
+			if (open) {
+				if (elements.size() > 0) {
+					ArrayList<E> result = new ArrayList<>(elements);
+					elements.clear();
+					return result;
+				} else {
+					return null;
+				}
+			} else {
+				throw new IllegalStateException("queue is closed");
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Returns the next element in the queue. If the queue is empty, this method
+	 * waits until at least one element is added.
+	 * 
+	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
+	 * Checking whether the queue is open and removing the next element is one atomic operation.
+	 * 
+	 * @return The next element in the queue, never null.
+	 * 
+	 * @throws IllegalStateException Thrown, if the queue is closed.
+	 * @throws InterruptedException Throw, if the thread is interrupted while waiting for an
+	 *                              element to be added.
+	 */
+	public E getElementBlocking() throws InterruptedException {
+		lock.lock();
+		try {
+			while (open && elements.isEmpty()) {
+				nonEmpty.await();
+			}
+			
+			if (open) {
+				return elements.removeFirst();
+			} else {
+				throw new IllegalStateException("queue is closed");
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Returns the next element in the queue. If the queue is empty, this method
+	 * waits at most a certain time until an element becomes available. If no element
+	 * is available after that time, the method returns null.
+	 * 
+	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
+	 * Checking whether the queue is open and removing the next element is one atomic operation.
+	 * 
+	 * @param timeoutMillis The number of milliseconds to block, at most.
+	 * @return The next element in the queue, or null, if the timeout expires  before an element is available.
+	 * 
+	 * @throws IllegalStateException Thrown, if the queue is closed.
+	 * @throws InterruptedException Throw, if the thread is interrupted while waiting for an
+	 *                              element to be added.
+	 */
+	public E getElementBlocking(long timeoutMillis) throws InterruptedException {
+		if (timeoutMillis == 0L) {
+			// wait forever case
+			return getElementBlocking();
+		} else if (timeoutMillis < 0L) {
+			throw new IllegalArgumentException("invalid timeout");
+		}
+		
+		final long deadline = System.currentTimeMillis() + timeoutMillis;
+		
+		lock.lock();
+		try {
+			while (open && elements.isEmpty() && timeoutMillis > 0) { 
+				nonEmpty.await(timeoutMillis, TimeUnit.MILLISECONDS);
+				timeoutMillis = deadline - System.currentTimeMillis();
+			}
+			
+			if (!open) {
+				throw new IllegalStateException("queue is closed");
+			}
+			else if (elements.isEmpty()) {
+				return null;
+			} else {
+				return elements.removeFirst();
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Gets all the elements found in the list, or blocks until at least one element
+	 * was added. If the queue is empty when this method is called, it blocks until
+	 * at least one element is added.
+	 *
+	 * <p>This method always returns a list with at least one element.
+	 * 
+	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
+	 * Checking whether the queue is open and removing the next element is one atomic operation.
+	 * 
+	 * @return A list with all elements in the queue, always at least one element.
+	 * 
+	 * @throws IllegalStateException Thrown, if the queue is closed.
+	 * @throws InterruptedException Throw, if the thread is interrupted while waiting for an
+	 *                              element to be added.
+	 */
+	public List<E> getBatchBlocking() throws InterruptedException {
+		lock.lock();
+		try {
+			while (open && elements.isEmpty()) {
+				nonEmpty.await();
+			}
+			if (open) {
+				ArrayList<E> result = new ArrayList<>(elements);
+				elements.clear();
+				return result;
+			} else {
+				throw new IllegalStateException("queue is closed");
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Gets all the elements found in the list, or blocks until at least one element
+	 * was added. This method is similar as {@link #getBatchBlocking()}, but takes
+	 * a number of milliseconds that the method will maximally wait before returning.
+	 * 
+	 * <p>This method never returns null, but an empty list, if the queue is empty when
+	 * the method is called and the request times out before an element was added.
+	 * 
+	 * <p>The method throws an {@code IllegalStateException} if the queue is closed.
+	 * Checking whether the queue is open and removing the next element is one atomic operation.
+	 * 
+	 * @param timeoutMillis The number of milliseconds to wait, at most.
+	 * @return A list with all elements in the queue, possible an empty list.
+	 *
+	 * @throws IllegalStateException Thrown, if the queue is closed.
+	 * @throws InterruptedException Throw, if the thread is interrupted while waiting for an
+	 *                              element to be added.
+	 */
+	public List<E> getBatchBlocking(long timeoutMillis) throws InterruptedException {
+		if (timeoutMillis == 0L) {
+			// wait forever case
+			return getBatchBlocking();
+		} else if (timeoutMillis < 0L) {
+			throw new IllegalArgumentException("invalid timeout");
+		}
+
+		final long deadline = System.currentTimeMillis() + timeoutMillis;
+
+		lock.lock();
+		try {
+			while (open && elements.isEmpty() && timeoutMillis > 0) {
+				nonEmpty.await(timeoutMillis, TimeUnit.MILLISECONDS);
+				timeoutMillis = deadline - System.currentTimeMillis();
+			}
+
+			if (!open) {
+				throw new IllegalStateException("queue is closed");
+			}
+			else if (elements.isEmpty()) {
+				return Collections.emptyList();
+			}
+			else {
+				ArrayList<E> result = new ArrayList<>(elements);
+				elements.clear();
+				return result;
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Standard Utilities
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public int hashCode() {
+		int hashCode = 17;
+		for (E element : elements) {
+			hashCode = 31 * hashCode + element.hashCode();
+		}
+		return hashCode;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		} else if (obj != null && obj.getClass() == ClosableBlockingQueue.class) {
+			@SuppressWarnings("unchecked")
+			ClosableBlockingQueue<E> that = (ClosableBlockingQueue<E>) obj;
+			
+			if (this.elements.size() == that.elements.size()) {
+				Iterator<E> thisElements = this.elements.iterator();
+				for (E thatNext : that.elements) {
+					E thisNext = thisElements.next();
+					if (!(thisNext == null ? thatNext == null : thisNext.equals(thatNext))) {
+						return false;
+					}
+				}
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public String toString() {
+		return elements.toString();
+	}
+}


[39/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
new file mode 100644
index 0000000..80ae294
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
@@ -0,0 +1,991 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.fs;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Type;
+import org.apache.avro.file.DataFileConstants;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.GenericData.StringType;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.testutils.MultiShotLatch;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.connectors.fs.AvroKeyValueSinkWriter.AvroKeyValue;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.NetUtils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Tests for {@link RollingSink}. These
+ * tests test the different output methods as well as the rolling feature using a manual clock
+ * that increases time in lockstep with element computation using latches.
+ *
+ * <p>
+ * This only tests the rolling behaviour of the sink. There is a separate ITCase that verifies
+ * exactly once behaviour.
+ *
+ * @deprecated should be removed with the {@link RollingSink}.
+ */
+@Deprecated
+public class RollingSinkITCase extends StreamingMultipleProgramsTestBase {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(RollingSinkITCase.class);
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	protected static MiniDFSCluster hdfsCluster;
+	protected static org.apache.hadoop.fs.FileSystem dfs;
+	protected static String hdfsURI;
+	protected static Configuration conf = new Configuration();
+
+	protected static File dataDir;
+
+	@BeforeClass
+	public static void createHDFS() throws IOException {
+
+		LOG.info("In RollingSinkITCase: Starting MiniDFSCluster ");
+
+		dataDir = tempFolder.newFolder();
+
+		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
+		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+		hdfsCluster = builder.build();
+
+		dfs = hdfsCluster.getFileSystem();
+
+		hdfsURI = "hdfs://"
+				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
+				+ "/";
+	}
+
+	@AfterClass
+	public static void destroyHDFS() {
+		LOG.info("In RollingSinkITCase: tearing down MiniDFSCluster ");
+		hdfsCluster.shutdown();
+	}
+
+	/**
+	 * This tests {@link StringWriter} with
+	 * non-rolling output.
+	 */
+	@Test
+	public void testNonRollingStringWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/string-non-rolling-out";
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
+				.broadcast()
+				.filter(new OddEvenFilter());
+
+		RollingSink<String> sink = new RollingSink<String>(outPath)
+				.setBucketer(new NonRollingBucketer())
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		source
+				.map(new MapFunction<Tuple2<Integer,String>, String>() {
+					private static final long serialVersionUID = 1L;
+					@Override
+					public String map(Tuple2<Integer, String> value) throws Exception {
+						return value.f1;
+					}
+				})
+				.addSink(sink);
+
+		env.execute("RollingSink String Write Test");
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
+
+		BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
+			String line = br.readLine();
+			Assert.assertEquals("message #" + i, line);
+		}
+
+		inStream.close();
+
+		inStream = dfs.open(new Path(outPath + "/part-1-0"));
+
+		br = new BufferedReader(new InputStreamReader(inStream));
+
+		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
+			String line = br.readLine();
+			Assert.assertEquals("message #" + i, line);
+		}
+
+		inStream.close();
+	}
+
+	/**
+	 * This tests {@link SequenceFileWriter}
+	 * with non-rolling output and without compression.
+	 */
+	@Test
+	public void testNonRollingSequenceFileWithoutCompressionWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/seq-no-comp-non-rolling-out";
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
+				.broadcast()
+				.filter(new OddEvenFilter());
+
+		DataStream<Tuple2<IntWritable, Text>> mapped =  source.map(new MapFunction<Tuple2<Integer,String>, Tuple2<IntWritable, Text>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public Tuple2<IntWritable, Text> map(Tuple2<Integer, String> value) throws Exception {
+				return Tuple2.of(new IntWritable(value.f0), new Text(value.f1));
+			}
+		});
+
+
+		RollingSink<Tuple2<IntWritable, Text>> sink = new RollingSink<Tuple2<IntWritable, Text>>(outPath)
+				.setWriter(new SequenceFileWriter<IntWritable, Text>())
+				.setBucketer(new NonRollingBucketer())
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		mapped.addSink(sink);
+
+		env.execute("RollingSink String Write Test");
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
+
+		SequenceFile.Reader reader = new SequenceFile.Reader(inStream,
+				1000,
+				0,
+				100000,
+				new Configuration());
+
+		IntWritable intWritable = new IntWritable();
+		Text txt = new Text();
+
+		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
+			reader.next(intWritable, txt);
+			Assert.assertEquals(i, intWritable.get());
+			Assert.assertEquals("message #" + i, txt.toString());
+		}
+
+		reader.close();
+		inStream.close();
+
+		inStream = dfs.open(new Path(outPath + "/part-1-0"));
+
+		reader = new SequenceFile.Reader(inStream,
+				1000,
+				0,
+				100000,
+				new Configuration());
+
+		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
+			reader.next(intWritable, txt);
+			Assert.assertEquals(i, intWritable.get());
+			Assert.assertEquals("message #" + i, txt.toString());
+		}
+
+		reader.close();
+		inStream.close();
+	}
+
+	/**
+	 * This tests {@link SequenceFileWriter}
+	 * with non-rolling output but with compression.
+	 */
+	@Test
+	public void testNonRollingSequenceFileWithCompressionWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/seq-non-rolling-out";
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
+				.broadcast()
+				.filter(new OddEvenFilter());
+
+		DataStream<Tuple2<IntWritable, Text>> mapped =  source.map(new MapFunction<Tuple2<Integer,String>, Tuple2<IntWritable, Text>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public Tuple2<IntWritable, Text> map(Tuple2<Integer, String> value) throws Exception {
+				return Tuple2.of(new IntWritable(value.f0), new Text(value.f1));
+			}
+		});
+
+
+		RollingSink<Tuple2<IntWritable, Text>> sink = new RollingSink<Tuple2<IntWritable, Text>>(outPath)
+				.setWriter(new SequenceFileWriter<IntWritable, Text>("Default", SequenceFile.CompressionType.BLOCK))
+				.setBucketer(new NonRollingBucketer())
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		mapped.addSink(sink);
+
+		env.execute("RollingSink String Write Test");
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
+
+		SequenceFile.Reader reader = new SequenceFile.Reader(inStream,
+				1000,
+				0,
+				100000,
+				new Configuration());
+
+		IntWritable intWritable = new IntWritable();
+		Text txt = new Text();
+
+		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
+			reader.next(intWritable, txt);
+			Assert.assertEquals(i, intWritable.get());
+			Assert.assertEquals("message #" + i, txt.toString());
+		}
+
+		reader.close();
+		inStream.close();
+
+		inStream = dfs.open(new Path(outPath + "/part-1-0"));
+
+		reader = new SequenceFile.Reader(inStream,
+				1000,
+				0,
+				100000,
+				new Configuration());
+
+		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
+			reader.next(intWritable, txt);
+			Assert.assertEquals(i, intWritable.get());
+			Assert.assertEquals("message #" + i, txt.toString());
+		}
+
+		reader.close();
+		inStream.close();
+	}
+	
+	
+	/**
+	 * This tests {@link AvroKeyValueSinkWriter}
+	 * with non-rolling output and without compression.
+	 */
+	@Test
+	public void testNonRollingAvroKeyValueWithoutCompressionWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/avro-kv-no-comp-non-rolling-out";
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
+				.broadcast()
+				.filter(new OddEvenFilter());
+
+
+		Map<String, String> properties = new HashMap<>();
+		Schema keySchema = Schema.create(Type.INT);
+		Schema valueSchema = Schema.create(Type.STRING);
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema.toString());
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema.toString());
+		RollingSink<Tuple2<Integer, String>> sink = new RollingSink<Tuple2<Integer, String>>(outPath)
+				.setWriter(new AvroKeyValueSinkWriter<Integer, String>(properties))
+				.setBucketer(new NonRollingBucketer())
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		source.addSink(sink);
+
+		env.execute("RollingSink Avro KeyValue Writer Test");
+
+		GenericData.setStringType(valueSchema, StringType.String);
+		Schema elementSchema = AvroKeyValue.getSchema(keySchema, valueSchema);
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
+		SpecificDatumReader<GenericRecord> elementReader = new SpecificDatumReader<GenericRecord>(elementSchema);
+		DataFileStream<GenericRecord> dataFileStream = new DataFileStream<GenericRecord>(inStream, elementReader);
+		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
+			AvroKeyValue<Integer, String> wrappedEntry = new AvroKeyValue<Integer, String>(dataFileStream.next());
+			int key = wrappedEntry.getKey().intValue();
+			Assert.assertEquals(i, key);
+			String value = wrappedEntry.getValue();
+			Assert.assertEquals("message #" + i, value);
+		}
+
+		dataFileStream.close();
+		inStream.close();
+
+		inStream = dfs.open(new Path(outPath + "/part-1-0"));
+		dataFileStream = new DataFileStream<GenericRecord>(inStream, elementReader);
+
+		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
+			AvroKeyValue<Integer, String> wrappedEntry = new AvroKeyValue<Integer, String>(dataFileStream.next());
+			int key = wrappedEntry.getKey().intValue();
+			Assert.assertEquals(i, key);
+			String value = wrappedEntry.getValue();
+			Assert.assertEquals("message #" + i, value);
+		}
+
+		dataFileStream.close();
+		inStream.close();
+	}
+	
+	/**
+	 * This tests {@link AvroKeyValueSinkWriter}
+	 * with non-rolling output and with compression.
+	 */
+	@Test
+	public void testNonRollingAvroKeyValueWithCompressionWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/avro-kv-no-comp-non-rolling-out";
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
+				.broadcast()
+				.filter(new OddEvenFilter());
+
+
+		Map<String, String> properties = new HashMap<>();
+		Schema keySchema = Schema.create(Type.INT);
+		Schema valueSchema = Schema.create(Type.STRING);
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema.toString());
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema.toString());
+		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS, String.valueOf(true));
+		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS_CODEC, DataFileConstants.SNAPPY_CODEC);
+		RollingSink<Tuple2<Integer, String>> sink = new RollingSink<Tuple2<Integer, String>>(outPath)
+				.setWriter(new AvroKeyValueSinkWriter<Integer, String>(properties))
+				.setBucketer(new NonRollingBucketer())
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		source.addSink(sink);
+
+		env.execute("RollingSink Avro KeyValue Writer Test");
+
+		GenericData.setStringType(valueSchema, StringType.String);
+		Schema elementSchema = AvroKeyValue.getSchema(keySchema, valueSchema);
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
+		SpecificDatumReader<GenericRecord> elementReader = new SpecificDatumReader<GenericRecord>(elementSchema);
+		DataFileStream<GenericRecord> dataFileStream = new DataFileStream<GenericRecord>(inStream, elementReader);
+		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
+			AvroKeyValue<Integer, String> wrappedEntry = new AvroKeyValue<Integer, String>(dataFileStream.next());
+			int key = wrappedEntry.getKey().intValue();
+			Assert.assertEquals(i, key);
+			String value = wrappedEntry.getValue();
+			Assert.assertEquals("message #" + i, value);
+		}
+
+		dataFileStream.close();
+		inStream.close();
+
+		inStream = dfs.open(new Path(outPath + "/part-1-0"));
+		dataFileStream = new DataFileStream<GenericRecord>(inStream, elementReader);
+
+		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
+			AvroKeyValue<Integer, String> wrappedEntry = new AvroKeyValue<Integer, String>(dataFileStream.next());
+			int key = wrappedEntry.getKey().intValue();
+			Assert.assertEquals(i, key);
+			String value = wrappedEntry.getValue();
+			Assert.assertEquals("message #" + i, value);
+		}
+
+		dataFileStream.close();
+		inStream.close();
+	}
+
+
+	/**
+	 * This tests user defined hdfs configuration
+	 * @throws Exception
+     */
+	@Test
+	public void testUserDefinedConfiguration() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/string-non-rolling-with-config";
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
+			.broadcast()
+			.filter(new OddEvenFilter());
+
+		Configuration conf = new Configuration();
+		conf.set("io.file.buffer.size", "40960");
+		RollingSink<String> sink = new RollingSink<String>(outPath)
+			.setFSConfig(conf)
+			.setWriter(new StreamWriterWithConfigCheck<String>("io.file.buffer.size", "40960"))
+			.setBucketer(new NonRollingBucketer())
+			.setPartPrefix("part")
+			.setPendingPrefix("")
+			.setPendingSuffix("");
+
+		source
+			.map(new MapFunction<Tuple2<Integer,String>, String>() {
+				private static final long serialVersionUID = 1L;
+				@Override
+				public String map(Tuple2<Integer, String> value) throws Exception {
+					return value.f1;
+				}
+			})
+			.addSink(sink);
+
+		env.execute("RollingSink with configuration Test");
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
+
+		BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
+			String line = br.readLine();
+			Assert.assertEquals("message #" + i, line);
+		}
+
+		inStream.close();
+
+		inStream = dfs.open(new Path(outPath + "/part-1-0"));
+
+		br = new BufferedReader(new InputStreamReader(inStream));
+
+		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
+			String line = br.readLine();
+			Assert.assertEquals("message #" + i, line);
+		}
+
+		inStream.close();
+	}
+
+	// we use this to synchronize the clock changes to elements being processed
+	final static MultiShotLatch latch1 = new MultiShotLatch();
+	final static MultiShotLatch latch2 = new MultiShotLatch();
+
+	/**
+	 * This uses {@link org.apache.flink.streaming.connectors.fs.DateTimeBucketer} to
+	 * produce rolling files. The clock of DateTimeBucketer is set to
+	 * {@link ModifyableClock} to keep the time in lockstep with the processing of elements using
+	 * latches.
+	 */
+	@Test
+	public void testDateTimeRollingStringWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/rolling-out";
+		DateTimeBucketer.setClock(new ModifyableClock());
+		ModifyableClock.setCurrentTime(0);
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new WaitingTestSourceFunction(
+				NUM_ELEMENTS))
+				.broadcast();
+
+		// the parallel flatMap is chained to the sink, so when it has seen 5 elements it can
+		// fire the latch
+		DataStream<String> mapped = source
+				.flatMap(new RichFlatMapFunction<Tuple2<Integer, String>, String>() {
+					private static final long serialVersionUID = 1L;
+
+					int count = 0;
+					@Override
+					public void flatMap(Tuple2<Integer, String> value,
+							Collector<String> out) throws Exception {
+						out.collect(value.f1);
+						count++;
+						if (count >= 5) {
+							if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
+								latch1.trigger();
+							} else {
+								latch2.trigger();
+							}
+							count = 0;
+						}
+					}
+
+				});
+
+		RollingSink<String> sink = new RollingSink<String>(outPath)
+				.setBucketer(new DateTimeBucketer("ss"))
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		mapped.addSink(sink);
+
+		env.execute("RollingSink String Write Test");
+
+		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(outPath), true);
+
+		// we should have 8 rolling files, 4 time intervals and parallelism of 2
+		int numFiles = 0;
+		while (files.hasNext()) {
+			LocatedFileStatus file = files.next();
+			numFiles++;
+			if (file.getPath().toString().contains("rolling-out/00")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 0; i < 5; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else if (file.getPath().toString().contains("rolling-out/05")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 5; i < 10; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else if (file.getPath().toString().contains("rolling-out/10")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 10; i < 15; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else if (file.getPath().toString().contains("rolling-out/15")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 15; i < 20; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else {
+				Assert.fail("File " + file + " does not match any expected roll pattern.");
+			}
+		}
+
+		Assert.assertEquals(8, numFiles);
+	}
+
+	private static final String PART_PREFIX = "part";
+	private static final String PENDING_SUFFIX = ".pending";
+	private static final String IN_PROGRESS_SUFFIX = ".in-progress";
+	private static final String VALID_LENGTH_SUFFIX = ".valid";
+
+	@Test
+	public void testBucketStateTransitions() throws Exception {
+		final File outDir = tempFolder.newFolder();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createRescalingTestSink(outDir, 1, 0);
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.setProcessingTime(0L);
+
+		// we have a bucket size of 5 bytes, so each record will get its own bucket,
+		// i.e. the bucket should roll after every record.
+
+		testHarness.processElement(new StreamRecord<>("test1", 1L));
+		testHarness.processElement(new StreamRecord<>("test2", 1L));
+		checkFs(outDir, 1, 1 ,0, 0);
+
+		testHarness.processElement(new StreamRecord<>("test3", 1L));
+		checkFs(outDir, 1, 2, 0, 0);
+
+		testHarness.snapshot(0, 0);
+		checkFs(outDir, 1, 2, 0, 0);
+
+		testHarness.notifyOfCompletedCheckpoint(0);
+		checkFs(outDir, 1, 0, 2, 0);
+
+		OperatorStateHandles snapshot = testHarness.snapshot(1, 0);
+
+		testHarness.close();
+		checkFs(outDir, 0, 1, 2, 0);
+
+		testHarness = createRescalingTestSink(outDir, 1, 0);
+		testHarness.setup();
+		testHarness.initializeState(snapshot);
+		testHarness.open();
+		checkFs(outDir, 0, 0, 3, 1);
+
+		snapshot = testHarness.snapshot(2, 0);
+
+		testHarness.processElement(new StreamRecord<>("test4", 10));
+		checkFs(outDir, 1, 0, 3, 1);
+
+		testHarness = createRescalingTestSink(outDir, 1, 0);
+		testHarness.setup();
+		testHarness.initializeState(snapshot);
+		testHarness.open();
+
+		// the in-progress file remains as we do not clean up now
+		checkFs(outDir, 1, 0, 3, 1);
+
+		testHarness.close();
+
+		// at close it is not moved to final because it is not part
+		// of the current task's state, it was just a not cleaned up leftover.
+		checkFs(outDir, 1, 0, 3, 1);
+	}
+
+	@Test
+	public void testScalingDown() throws Exception {
+		final File outDir = tempFolder.newFolder();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 3, 0);
+		testHarness1.setup();
+		testHarness1.open();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 3, 1);
+		testHarness2.setup();
+		testHarness2.open();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness3 = createRescalingTestSink(outDir, 3, 2);
+		testHarness3.setup();
+		testHarness3.open();
+
+		testHarness1.processElement(new StreamRecord<>("test1", 0L));
+		checkFs(outDir, 1, 0, 0, 0);
+
+		testHarness2.processElement(new StreamRecord<>("test2", 0L));
+		testHarness2.processElement(new StreamRecord<>("test3", 0L));
+		testHarness2.processElement(new StreamRecord<>("test4", 0L));
+		testHarness2.processElement(new StreamRecord<>("test5", 0L));
+		testHarness2.processElement(new StreamRecord<>("test6", 0L));
+		checkFs(outDir, 2, 4, 0, 0);
+
+		testHarness3.processElement(new StreamRecord<>("test7", 0L));
+		testHarness3.processElement(new StreamRecord<>("test8", 0L));
+		checkFs(outDir, 3, 5, 0, 0);
+
+		// intentionally we snapshot them in a not ascending order so that the states are shuffled
+		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
+			testHarness3.snapshot(0, 0),
+			testHarness1.snapshot(0, 0),
+			testHarness2.snapshot(0, 0)
+		);
+
+		// with the above state reshuffling, we expect testHarness4 to take the
+		// state of the previous testHarness3 and testHarness1 while testHarness5
+		// will take that of the previous testHarness1
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness4 = createRescalingTestSink(outDir, 2, 0);
+		testHarness4.setup();
+		testHarness4.initializeState(mergedSnapshot);
+		testHarness4.open();
+
+		// we do not have a length file for part-2-0 because bucket part-2-0
+		// was not "in-progress", but "pending" (its full content is valid).
+		checkFs(outDir, 1, 4, 3, 2);
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness5 = createRescalingTestSink(outDir, 2, 1);
+		testHarness5.setup();
+		testHarness5.initializeState(mergedSnapshot);
+		testHarness5.open();
+
+		checkFs(outDir, 0, 0, 8, 3);
+	}
+
+	@Test
+	public void testScalingUp() throws Exception {
+		final File outDir = tempFolder.newFolder();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 2, 0);
+		testHarness1.setup();
+		testHarness1.open();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 2, 0);
+		testHarness2.setup();
+		testHarness2.open();
+
+		testHarness1.processElement(new StreamRecord<>("test1", 0L));
+		testHarness1.processElement(new StreamRecord<>("test2", 0L));
+
+		checkFs(outDir, 1, 1, 0, 0);
+
+		testHarness2.processElement(new StreamRecord<>("test3", 0L));
+		testHarness2.processElement(new StreamRecord<>("test4", 0L));
+		testHarness2.processElement(new StreamRecord<>("test5", 0L));
+
+		checkFs(outDir, 2, 3, 0, 0);
+
+		// intentionally we snapshot them in the reverse order so that the states are shuffled
+		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
+			testHarness2.snapshot(0, 0),
+			testHarness1.snapshot(0, 0)
+		);
+
+		testHarness1 = createRescalingTestSink(outDir, 3, 0);
+		testHarness1.setup();
+		testHarness1.initializeState(mergedSnapshot);
+		testHarness1.open();
+
+		checkFs(outDir, 1, 1, 3, 1);
+
+		testHarness2 = createRescalingTestSink(outDir, 3, 1);
+		testHarness2.setup();
+		testHarness2.initializeState(mergedSnapshot);
+		testHarness2.open();
+
+		checkFs(outDir, 0, 0, 5, 2);
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness3 = createRescalingTestSink(outDir, 3, 2);
+		testHarness3.setup();
+		testHarness3.initializeState(mergedSnapshot);
+		testHarness3.open();
+
+		checkFs(outDir, 0, 0, 5, 2);
+
+		testHarness1.processElement(new StreamRecord<>("test6", 0));
+		testHarness2.processElement(new StreamRecord<>("test6", 0));
+		testHarness3.processElement(new StreamRecord<>("test6", 0));
+
+		// 3 for the different tasks
+		checkFs(outDir, 3, 0, 5, 2);
+
+		testHarness1.snapshot(1, 0);
+		testHarness2.snapshot(1, 0);
+		testHarness3.snapshot(1, 0);
+
+		testHarness1.close();
+		testHarness2.close();
+		testHarness3.close();
+
+		checkFs(outDir, 0, 3, 5, 2);
+	}
+
+	private OneInputStreamOperatorTestHarness<String, Object> createRescalingTestSink(
+		File outDir, int totalParallelism, int taskIdx) throws Exception {
+
+		RollingSink<String> sink = new RollingSink<String>(outDir.getAbsolutePath())
+			.setWriter(new StringWriter<String>())
+			.setBatchSize(5)
+			.setPartPrefix(PART_PREFIX)
+			.setInProgressPrefix("")
+			.setPendingPrefix("")
+			.setValidLengthPrefix("")
+			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
+			.setPendingSuffix(PENDING_SUFFIX)
+			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
+
+		return createTestSink(sink, totalParallelism, taskIdx);
+	}
+
+	private <T> OneInputStreamOperatorTestHarness<T, Object> createTestSink(
+		RollingSink<T> sink, int totalParallelism, int taskIdx) throws Exception {
+		return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), 10, totalParallelism, taskIdx);
+	}
+
+	private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
+		int inProg = 0;
+		int pend = 0;
+		int compl = 0;
+		int val = 0;
+
+		for (File file: FileUtils.listFiles(outDir, null, true)) {
+			if (file.getAbsolutePath().endsWith("crc")) {
+				continue;
+			}
+			String path = file.getPath();
+			if (path.endsWith(IN_PROGRESS_SUFFIX)) {
+				inProg++;
+			} else if (path.endsWith(PENDING_SUFFIX)) {
+				pend++;
+			} else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
+				val++;
+			} else if (path.contains(PART_PREFIX)) {
+				compl++;
+			}
+		}
+
+		Assert.assertEquals(inprogress, inProg);
+		Assert.assertEquals(pending, pend);
+		Assert.assertEquals(completed, compl);
+		Assert.assertEquals(valid, val);
+	}
+
+	private static class TestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		private final int numElements;
+
+		public TestSourceFunction(int numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
+			for (int i = 0; i < numElements && running; i++) {
+				ctx.collect(Tuple2.of(i, "message #" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+	/**
+	 * This waits on the two multi-shot latches. The latches are triggered in a parallel
+	 * flatMap inside the test topology.
+	 */
+	private static class WaitingTestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		private final int numElements;
+
+		public WaitingTestSourceFunction(int numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
+			for (int i = 0; i < numElements && running; i++) {
+				if (i % 5 == 0 && i > 0) {
+					// update the clock after "five seconds", so we get 20 seconds in total
+					// with 5 elements in each time window
+					latch1.await();
+					latch2.await();
+					ModifyableClock.setCurrentTime(i * 1000);
+				}
+				ctx.collect(Tuple2.of(i, "message #" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+
+	private static class StreamWriterWithConfigCheck<T> extends StringWriter<T> {
+		private String key;
+		private String expect;
+		public StreamWriterWithConfigCheck(String key, String expect) {
+			this.key = key;
+			this.expect = expect;
+		}
+
+		@Override
+		public void open(FileSystem fs, Path path) throws IOException {
+			super.open(fs, path);
+			Assert.assertEquals(expect, fs.getConf().get(key));
+		}
+
+		@Override
+		public Writer<T> duplicate() {
+			return new StreamWriterWithConfigCheck<>(key, expect);
+		}
+	}
+
+	public static class OddEvenFilter extends RichFilterFunction<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(Tuple2<Integer, String> value) throws Exception {
+			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
+				return value.f0 % 2 == 0;
+			} else {
+				return value.f0 % 2 == 1;
+			}
+		}
+	}
+
+	public static class ModifyableClock implements Clock {
+
+		private static volatile long currentTime = 0;
+
+		public static void setCurrentTime(long currentTime) {
+			ModifyableClock.currentTime = currentTime;
+		}
+
+		@Override
+		public long currentTimeMillis() {
+			return currentTime;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
new file mode 100644
index 0000000..eb12d07
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.fs;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.security.SecurityUtils;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
+import org.apache.flink.test.util.SecureTestEnvironment;
+import org.apache.flink.test.util.TestingSecurityContext;
+import org.apache.flink.test.util.TestBaseUtils;
+import org.apache.flink.util.NetUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.VersionInfo;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTP_POLICY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
+
+/**
+ * Tests for running {@link RollingSinkSecuredITCase} which is an extension of {@link RollingSink} in secure environment
+ * Note: only executed for Hadoop version > 3.x.x
+ */
+public class RollingSinkSecuredITCase extends RollingSinkITCase {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(RollingSinkSecuredITCase.class);
+
+	/**
+	 * Skips all tests if the Hadoop version doesn't match.
+	 * We can't run this test class until HDFS-9213 is fixed which allows a secure DataNode
+	 * to bind to non-privileged ports for testing.
+	 * For now, we skip this test class until Hadoop version 3.x.x.
+	 */
+	private static void skipIfHadoopVersionIsNotAppropriate() {
+		// Skips all tests if the Hadoop version doesn't match
+		String hadoopVersionString = VersionInfo.getVersion();
+		String[] split = hadoopVersionString.split("\\.");
+		if (split.length != 3) {
+			throw new IllegalStateException("Hadoop version was not of format 'X.X.X': " + hadoopVersionString);
+		}
+		Assume.assumeTrue(
+			// check whether we're running Hadoop version >= 3.x.x
+			Integer.parseInt(split[0]) >= 3
+		);
+	}
+
+	/*
+	 * override super class static methods to avoid creating MiniDFS and MiniFlink with wrong configurations
+	 * and out-of-order sequence for secure cluster
+	 */
+	@BeforeClass
+	public static void setup() throws Exception {}
+
+	@AfterClass
+	public static void teardown() throws Exception {}
+
+	@BeforeClass
+	public static void createHDFS() throws IOException {}
+
+	@AfterClass
+	public static void destroyHDFS() {}
+
+	@BeforeClass
+	public static void startSecureCluster() throws Exception {
+
+		skipIfHadoopVersionIsNotAppropriate();
+
+		LOG.info("starting secure cluster environment for testing");
+
+		dataDir = tempFolder.newFolder();
+
+		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
+
+		SecureTestEnvironment.prepare(tempFolder);
+
+		populateSecureConfigurations();
+
+		Configuration flinkConfig = new Configuration();
+		flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY,
+				SecureTestEnvironment.getTestKeytab());
+		flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY,
+				SecureTestEnvironment.getHadoopServicePrincipal());
+
+		SecurityUtils.SecurityConfiguration ctx = new SecurityUtils.SecurityConfiguration(flinkConfig);
+		ctx.setHadoopConfiguration(conf);
+		try {
+			TestingSecurityContext.install(ctx, SecureTestEnvironment.getClientSecurityConfigurationMap());
+		} catch (Exception e) {
+			throw new RuntimeException("Exception occurred while setting up secure test context. Reason: {}", e);
+		}
+
+		File hdfsSiteXML = new File(dataDir.getAbsolutePath() + "/hdfs-site.xml");
+
+		FileWriter writer = new FileWriter(hdfsSiteXML);
+		conf.writeXml(writer);
+		writer.flush();
+		writer.close();
+
+		Map<String, String> map = new HashMap<String, String>(System.getenv());
+		map.put("HADOOP_CONF_DIR", hdfsSiteXML.getParentFile().getAbsolutePath());
+		TestBaseUtils.setEnv(map);
+
+
+		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+		builder.checkDataNodeAddrConfig(true);
+		builder.checkDataNodeHostConfig(true);
+		hdfsCluster = builder.build();
+
+		dfs = hdfsCluster.getFileSystem();
+
+		hdfsURI = "hdfs://"
+				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
+				+ "/";
+
+		startSecureFlinkClusterWithRecoveryModeEnabled();
+	}
+
+	@AfterClass
+	public static void teardownSecureCluster() throws Exception {
+		LOG.info("tearing down secure cluster environment");
+
+		TestStreamEnvironment.unsetAsContext();
+		stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT);
+
+		if (hdfsCluster != null) {
+			hdfsCluster.shutdown();
+		}
+
+		SecureTestEnvironment.cleanup();
+	}
+
+	private static void populateSecureConfigurations() {
+
+		String dataTransferProtection = "authentication";
+
+		SecurityUtil.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
+		conf.set(DFS_NAMENODE_USER_NAME_KEY, SecureTestEnvironment.getHadoopServicePrincipal());
+		conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, SecureTestEnvironment.getTestKeytab());
+		conf.set(DFS_DATANODE_USER_NAME_KEY, SecureTestEnvironment.getHadoopServicePrincipal());
+		conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, SecureTestEnvironment.getTestKeytab());
+		conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, SecureTestEnvironment.getHadoopServicePrincipal());
+
+		conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
+
+		conf.set("dfs.data.transfer.protection", dataTransferProtection);
+
+		conf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTP_ONLY.name());
+
+		conf.set(DFS_ENCRYPT_DATA_TRANSFER_KEY, "false");
+
+		conf.setInt("dfs.datanode.socket.write.timeout", 0);
+
+		/*
+		 * We ae setting the port number to privileged port - see HDFS-9213
+		 * This requires the user to have root privilege to bind to the port
+		 * Use below command (ubuntu) to set privilege to java process for the
+		 * bind() to work if the java process is not running as root.
+		 * setcap 'cap_net_bind_service=+ep' /path/to/java
+		 */
+		conf.set(DFS_DATANODE_ADDRESS_KEY, "localhost:1002");
+		conf.set(DFS_DATANODE_HOST_NAME_KEY, "localhost");
+		conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "localhost:1003");
+	}
+
+	private static void startSecureFlinkClusterWithRecoveryModeEnabled() {
+		try {
+			LOG.info("Starting Flink and ZK in secure mode");
+
+			dfs.mkdirs(new Path("/flink/checkpoints"));
+			dfs.mkdirs(new Path("/flink/recovery"));
+
+			org.apache.flink.configuration.Configuration config = new org.apache.flink.configuration.Configuration();
+
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, DEFAULT_PARALLELISM);
+			config.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3);
+			config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+			config.setString(ConfigConstants.STATE_BACKEND, "filesystem");
+			config.setString(ConfigConstants.ZOOKEEPER_CHECKPOINTS_PATH, hdfsURI + "/flink/checkpoints");
+			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, hdfsURI + "/flink/recovery");
+			config.setString("state.backend.fs.checkpointdir", hdfsURI + "/flink/checkpoints");
+
+			SecureTestEnvironment.populateFlinkSecureConfigurations(config);
+
+			cluster = TestBaseUtils.startCluster(config, false);
+			TestStreamEnvironment.setAsContext(cluster, DEFAULT_PARALLELISM);
+
+		} catch (Exception e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+	/* For secure cluster testing, it is enough to run only one test and override below test methods
+	 * to keep the overall build time minimal
+	 */
+	@Override
+	public void testNonRollingSequenceFileWithoutCompressionWriter() throws Exception {}
+
+	@Override
+	public void testNonRollingSequenceFileWithCompressionWriter() throws Exception {}
+
+	@Override
+	public void testNonRollingAvroKeyValueWithoutCompressionWriter() throws Exception {}
+
+	@Override
+	public void testNonRollingAvroKeyValueWithCompressionWriter() throws Exception {}
+
+	@Override
+	public void testDateTimeRollingStringWriter() throws Exception {}
+
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
new file mode 100644
index 0000000..d671874
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
@@ -0,0 +1,867 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs.bucketing;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileConstants;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.connectors.fs.AvroKeyValueSinkWriter;
+import org.apache.flink.streaming.connectors.fs.Clock;
+import org.apache.flink.streaming.connectors.fs.SequenceFileWriter;
+import org.apache.flink.streaming.connectors.fs.StringWriter;
+import org.apache.flink.streaming.connectors.fs.Writer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.util.NetUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.HashMap;
+import java.util.Map;
+
+public class BucketingSinkTest {
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	private static MiniDFSCluster hdfsCluster;
+	private static org.apache.hadoop.fs.FileSystem dfs;
+	private static String hdfsURI;
+
+	private static final String PART_PREFIX = "part";
+	private static final String PENDING_SUFFIX = ".pending";
+	private static final String IN_PROGRESS_SUFFIX = ".in-progress";
+	private static final String VALID_LENGTH_SUFFIX = ".valid";
+
+	private OneInputStreamOperatorTestHarness<String, Object> createRescalingTestSink(
+		File outDir, int totalParallelism, int taskIdx, long inactivityInterval) throws Exception {
+
+		BucketingSink<String> sink = new BucketingSink<String>(outDir.getAbsolutePath())
+			.setBucketer(new Bucketer<String>() {
+				private static final long serialVersionUID = 1L;
+
+				@Override
+				public Path getBucketPath(Clock clock, Path basePath, String element) {
+					return new Path(basePath, element);
+				}
+			})
+			.setWriter(new StringWriter<String>())
+			.setInactiveBucketCheckInterval(inactivityInterval)
+			.setInactiveBucketThreshold(inactivityInterval)
+			.setPartPrefix(PART_PREFIX)
+			.setInProgressPrefix("")
+			.setPendingPrefix("")
+			.setValidLengthPrefix("")
+			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
+			.setPendingSuffix(PENDING_SUFFIX)
+			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
+
+		return createTestSink(sink, totalParallelism, taskIdx);
+	}
+
+	private OneInputStreamOperatorTestHarness<String, Object> createTestSink(File dataDir, int totalParallelism, int taskIdx) throws Exception {
+		BucketingSink<String> sink = new BucketingSink<String>(dataDir.getAbsolutePath())
+			.setBucketer(new Bucketer<String>() {
+				private static final long serialVersionUID = 1L;
+
+				@Override
+				public Path getBucketPath(Clock clock, Path basePath, String element) {
+					return new Path(basePath, element);
+				}
+			})
+			.setWriter(new StringWriter<String>())
+			.setPartPrefix(PART_PREFIX)
+			.setPendingPrefix("")
+			.setInactiveBucketCheckInterval(5*60*1000L)
+			.setInactiveBucketThreshold(5*60*1000L)
+			.setPendingSuffix(PENDING_SUFFIX)
+			.setInProgressSuffix(IN_PROGRESS_SUFFIX);
+
+		return createTestSink(sink, totalParallelism, taskIdx);
+	}
+
+	private <T> OneInputStreamOperatorTestHarness<T, Object> createTestSink(
+			BucketingSink<T> sink, int totalParallelism, int taskIdx) throws Exception {
+		return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), 10, totalParallelism, taskIdx);
+	}
+
+	@BeforeClass
+	public static void createHDFS() throws IOException {
+		Configuration conf = new Configuration();
+
+		File dataDir = tempFolder.newFolder();
+
+		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
+		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+		hdfsCluster = builder.build();
+
+		dfs = hdfsCluster.getFileSystem();
+
+		hdfsURI = "hdfs://"
+			+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
+			+ "/";
+	}
+
+	@AfterClass
+	public static void destroyHDFS() {
+		hdfsCluster.shutdown();
+	}
+
+	@Test
+	public void testInactivityPeriodWithLateNotify() throws Exception {
+		final File outDir = tempFolder.newFolder();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createRescalingTestSink(outDir, 1, 0, 100);
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.processElement(new StreamRecord<>("test1", 1L));
+		testHarness.processElement(new StreamRecord<>("test2", 1L));
+		checkFs(outDir, 2, 0 ,0, 0);
+
+		testHarness.setProcessingTime(101L);	// put some in pending
+		checkFs(outDir, 0, 2, 0, 0);
+
+		testHarness.snapshot(0, 0);				// put them in pending for 0
+		checkFs(outDir, 0, 2, 0, 0);
+
+		testHarness.processElement(new StreamRecord<>("test3", 1L));
+		testHarness.processElement(new StreamRecord<>("test4", 1L));
+
+		testHarness.setProcessingTime(202L);	// put some in pending
+
+		testHarness.snapshot(1, 0);				// put them in pending for 1
+		checkFs(outDir, 0, 4, 0, 0);
+
+		testHarness.notifyOfCompletedCheckpoint(0);	// put the pending for 0 to the "committed" state
+		checkFs(outDir, 0, 2, 2, 0);
+
+		testHarness.notifyOfCompletedCheckpoint(1); // put the pending for 1 to the "committed" state
+		checkFs(outDir, 0, 0, 4, 0);
+	}
+
+	@Test
+	public void testBucketStateTransitions() throws Exception {
+		final File outDir = tempFolder.newFolder();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createRescalingTestSink(outDir, 1, 0, 100);
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.processElement(new StreamRecord<>("test1", 1L));
+		testHarness.processElement(new StreamRecord<>("test2", 1L));
+		checkFs(outDir, 2, 0 ,0, 0);
+
+		// this is to check the inactivity threshold
+		testHarness.setProcessingTime(101L);
+		checkFs(outDir, 0, 2, 0, 0);
+
+		testHarness.processElement(new StreamRecord<>("test3", 1L));
+		checkFs(outDir, 1, 2, 0, 0);
+
+		testHarness.snapshot(0, 0);
+		checkFs(outDir, 1, 2, 0, 0);
+
+		testHarness.notifyOfCompletedCheckpoint(0);
+		checkFs(outDir, 1, 0, 2, 0);
+
+		OperatorStateHandles snapshot = testHarness.snapshot(1, 0);
+
+		testHarness.close();
+		checkFs(outDir, 0, 1, 2, 0);
+
+		testHarness = createRescalingTestSink(outDir, 1, 0, 100);
+		testHarness.setup();
+		testHarness.initializeState(snapshot);
+		testHarness.open();
+		checkFs(outDir, 0, 0, 3, 1);
+
+		snapshot = testHarness.snapshot(2, 0);
+
+		testHarness.processElement(new StreamRecord<>("test4", 10));
+		checkFs(outDir, 1, 0, 3, 1);
+
+		testHarness = createRescalingTestSink(outDir, 1, 0, 100);
+		testHarness.setup();
+		testHarness.initializeState(snapshot);
+		testHarness.open();
+
+		// the in-progress file remains as we do not clean up now
+		checkFs(outDir, 1, 0, 3, 1);
+
+		testHarness.close();
+
+		// at close it is not moved to final because it is not part
+		// of the current task's state, it was just a not cleaned up leftover.
+		checkFs(outDir, 1, 0, 3, 1);
+	}
+
+	@Test
+	public void testSameParallelismWithShufflingStates() throws Exception {
+		final File outDir = tempFolder.newFolder();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 2, 0, 100);
+		testHarness1.setup();
+		testHarness1.open();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 2, 1, 100);
+		testHarness2.setup();
+		testHarness2.open();
+
+		testHarness1.processElement(new StreamRecord<>("test1", 0L));
+		checkFs(outDir, 1, 0, 0, 0);
+
+		testHarness2.processElement(new StreamRecord<>("test2", 0L));
+		checkFs(outDir, 2, 0, 0, 0);
+
+		// intentionally we snapshot them in the reverse order so that the states are shuffled
+		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
+			testHarness2.snapshot(0, 0),
+			testHarness1.snapshot(0, 0)
+		);
+
+		checkFs(outDir, 2, 0, 0, 0);
+
+		// this will not be included in any checkpoint so it can be cleaned up (although we do not)
+		testHarness2.processElement(new StreamRecord<>("test3", 0L));
+		checkFs(outDir, 3, 0, 0, 0);
+
+		testHarness1 = createRescalingTestSink(outDir, 2, 0, 100);
+		testHarness1.setup();
+		testHarness1.initializeState(mergedSnapshot);
+		testHarness1.open();
+
+		// the one in-progress will be the one assigned to the next instance,
+		// the other is the test3 which is just not cleaned up
+		checkFs(outDir, 2, 0, 1, 1);
+
+		testHarness2 = createRescalingTestSink(outDir, 2, 1, 100);
+		testHarness2.setup();
+		testHarness2.initializeState(mergedSnapshot);
+		testHarness2.open();
+
+		checkFs(outDir, 1, 0, 2, 2);
+
+		testHarness1.close();
+		testHarness2.close();
+
+		// the 1 in-progress can be discarded.
+		checkFs(outDir, 1, 0, 2, 2);
+	}
+
+	@Test
+	public void testScalingDown() throws Exception {
+		final File outDir = tempFolder.newFolder();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 3, 0, 100);
+		testHarness1.setup();
+		testHarness1.open();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 3, 1, 100);
+		testHarness2.setup();
+		testHarness2.open();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness3 = createRescalingTestSink(outDir, 3, 2, 100);
+		testHarness3.setup();
+		testHarness3.open();
+
+		testHarness1.processElement(new StreamRecord<>("test1", 0L));
+		checkFs(outDir, 1, 0, 0, 0);
+
+		testHarness2.processElement(new StreamRecord<>("test2", 0L));
+		checkFs(outDir, 2, 0, 0, 0);
+
+		testHarness3.processElement(new StreamRecord<>("test3", 0L));
+		testHarness3.processElement(new StreamRecord<>("test4", 0L));
+		checkFs(outDir, 4, 0, 0, 0);
+
+		// intentionally we snapshot them in the reverse order so that the states are shuffled
+		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
+			testHarness3.snapshot(0, 0),
+			testHarness1.snapshot(0, 0),
+			testHarness2.snapshot(0, 0)
+		);
+
+		testHarness1 = createRescalingTestSink(outDir, 2, 0, 100);
+		testHarness1.setup();
+		testHarness1.initializeState(mergedSnapshot);
+		testHarness1.open();
+
+		checkFs(outDir, 1, 0, 3, 3);
+
+		testHarness2 = createRescalingTestSink(outDir, 2, 1, 100);
+		testHarness2.setup();
+		testHarness2.initializeState(mergedSnapshot);
+		testHarness2.open();
+
+		checkFs(outDir, 0, 0, 4, 4);
+	}
+
+	@Test
+	public void testScalingUp() throws Exception {
+		final File outDir = tempFolder.newFolder();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness1 = createRescalingTestSink(outDir, 2, 0, 100);
+		testHarness1.setup();
+		testHarness1.open();
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness2 = createRescalingTestSink(outDir, 2, 0, 100);
+		testHarness2.setup();
+		testHarness2.open();
+
+		testHarness1.processElement(new StreamRecord<>("test1", 1L));
+		testHarness1.processElement(new StreamRecord<>("test2", 1L));
+
+		checkFs(outDir, 2, 0, 0, 0);
+
+		testHarness2.processElement(new StreamRecord<>("test3", 1L));
+		testHarness2.processElement(new StreamRecord<>("test4", 1L));
+		testHarness2.processElement(new StreamRecord<>("test5", 1L));
+
+		checkFs(outDir, 5, 0, 0, 0);
+
+		// intentionally we snapshot them in the reverse order so that the states are shuffled
+		OperatorStateHandles mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState(
+			testHarness2.snapshot(0, 0),
+			testHarness1.snapshot(0, 0)
+		);
+
+		testHarness1 = createRescalingTestSink(outDir, 3, 0, 100);
+		testHarness1.setup();
+		testHarness1.initializeState(mergedSnapshot);
+		testHarness1.open();
+
+		checkFs(outDir, 2, 0, 3, 3);
+
+		testHarness2 = createRescalingTestSink(outDir, 3, 1, 100);
+		testHarness2.setup();
+		testHarness2.initializeState(mergedSnapshot);
+		testHarness2.open();
+
+		checkFs(outDir, 0, 0, 5, 5);
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness3 = createRescalingTestSink(outDir, 3, 2, 100);
+		testHarness3.setup();
+		testHarness3.initializeState(mergedSnapshot);
+		testHarness3.open();
+
+		checkFs(outDir, 0, 0, 5, 5);
+
+		testHarness1.processElement(new StreamRecord<>("test6", 0));
+		testHarness2.processElement(new StreamRecord<>("test6", 0));
+		testHarness3.processElement(new StreamRecord<>("test6", 0));
+
+		checkFs(outDir, 3, 0, 5, 5);
+
+		testHarness1.snapshot(1, 0);
+		testHarness2.snapshot(1, 0);
+		testHarness3.snapshot(1, 0);
+
+		testHarness1.close();
+		testHarness2.close();
+		testHarness3.close();
+
+		checkFs(outDir, 0, 3, 5, 5);
+	}
+
+	private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
+		int inProg = 0;
+		int pend = 0;
+		int compl = 0;
+		int val = 0;
+
+		for (File file: FileUtils.listFiles(outDir, null, true)) {
+			if (file.getAbsolutePath().endsWith("crc")) {
+				continue;
+			}
+			String path = file.getPath();
+			if (path.endsWith(IN_PROGRESS_SUFFIX)) {
+				inProg++;
+			} else if (path.endsWith(PENDING_SUFFIX)) {
+				pend++;
+			} else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
+				val++;
+			} else if (path.contains(PART_PREFIX)) {
+				compl++;
+			}
+		}
+
+		Assert.assertEquals(inprogress, inProg);
+		Assert.assertEquals(pending, pend);
+		Assert.assertEquals(completed, compl);
+		Assert.assertEquals(valid, val);
+	}
+
+	/**
+	 * This tests {@link StringWriter} with
+	 * non-bucketing output.
+	 */
+	@Test
+	public void testNonRollingStringWriter() throws Exception {
+		final String outPath = hdfsURI + "/string-non-rolling-out";
+
+		final int numElements = 20;
+
+		BucketingSink<String> sink = new BucketingSink<String>(outPath)
+			.setBucketer(new BasePathBucketer<String>())
+			.setPartPrefix(PART_PREFIX)
+			.setPendingPrefix("")
+			.setPendingSuffix("");
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(sink, 1, 0);
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.setup();
+		testHarness.open();
+
+		for (int i = 0; i < numElements; i++) {
+			testHarness.processElement(new StreamRecord<>("message #" + Integer.toString(i)));
+		}
+
+		testHarness.close();
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/" + PART_PREFIX + "-0-0"));
+
+		BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+		for (int i = 0; i < numElements; i++) {
+			String line = br.readLine();
+			Assert.assertEquals("message #" + i, line);
+		}
+
+		inStream.close();
+	}
+
+	/**
+	 * This tests {@link SequenceFileWriter}
+	 * with non-rolling output and without compression.
+	 */
+	@Test
+	public void testNonRollingSequenceFileWithoutCompressionWriter() throws Exception {
+		final String outPath = hdfsURI + "/seq-no-comp-non-rolling-out";
+
+		final int numElements = 20;
+
+		BucketingSink<Tuple2<IntWritable, Text>> sink = new BucketingSink<Tuple2<IntWritable, Text>>(outPath)
+			.setWriter(new SequenceFileWriter<IntWritable, Text>())
+			.setBucketer(new BasePathBucketer<Tuple2<IntWritable, Text>>())
+			.setPartPrefix(PART_PREFIX)
+			.setPendingPrefix("")
+			.setPendingSuffix("");
+
+		sink.setInputType(TypeInformation.of(new TypeHint<Tuple2<IntWritable, Text>>(){}), new ExecutionConfig());
+
+		OneInputStreamOperatorTestHarness<Tuple2<IntWritable, Text>, Object> testHarness =
+			createTestSink(sink, 1, 0);
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.setup();
+		testHarness.open();
+
+		for (int i = 0; i < numElements; i++) {
+			testHarness.processElement(new StreamRecord<>(Tuple2.of(
+				new IntWritable(i),
+				new Text("message #" + Integer.toString(i))
+			)));
+		}
+
+		testHarness.close();
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/" + PART_PREFIX + "-0-0"));
+
+		SequenceFile.Reader reader = new SequenceFile.Reader(inStream, 1000, 0, 100000, new Configuration());
+
+		IntWritable intWritable = new IntWritable();
+		Text txt = new Text();
+
+		for (int i = 0; i < numElements; i++) {
+			reader.next(intWritable, txt);
+			Assert.assertEquals(i, intWritable.get());
+			Assert.assertEquals("message #" + i, txt.toString());
+		}
+
+		reader.close();
+		inStream.close();
+	}
+
+	/**
+	 * This tests {@link AvroKeyValueSinkWriter}
+	 * with non-rolling output and with compression.
+	 */
+	@Test
+	public void testNonRollingAvroKeyValueWithCompressionWriter() throws Exception {
+		final String outPath = hdfsURI + "/avro-kv-no-comp-non-rolling-out";
+
+		final int numElements = 20;
+
+		Map<String, String> properties = new HashMap<>();
+		Schema keySchema = Schema.create(Schema.Type.INT);
+		Schema valueSchema = Schema.create(Schema.Type.STRING);
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema.toString());
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema.toString());
+		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS, String.valueOf(true));
+		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS_CODEC, DataFileConstants.SNAPPY_CODEC);
+
+		BucketingSink<Tuple2<Integer, String>> sink = new BucketingSink<Tuple2<Integer, String>>(outPath)
+			.setWriter(new AvroKeyValueSinkWriter<Integer, String>(properties))
+			.setBucketer(new BasePathBucketer<Tuple2<Integer, String>>())
+			.setPartPrefix(PART_PREFIX)
+			.setPendingPrefix("")
+			.setPendingSuffix("");
+
+		OneInputStreamOperatorTestHarness<Tuple2<Integer, String>, Object> testHarness =
+			createTestSink(sink, 1, 0);
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.setup();
+		testHarness.open();
+
+		for (int i = 0; i < numElements; i++) {
+			testHarness.processElement(new StreamRecord<>(Tuple2.of(
+				i, "message #" + Integer.toString(i)
+			)));
+		}
+
+		testHarness.close();
+
+		GenericData.setStringType(valueSchema, GenericData.StringType.String);
+		Schema elementSchema = AvroKeyValueSinkWriter.AvroKeyValue.getSchema(keySchema, valueSchema);
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/" + PART_PREFIX + "-0-0"));
+
+		SpecificDatumReader<GenericRecord> elementReader = new SpecificDatumReader<>(elementSchema);
+		DataFileStream<GenericRecord> dataFileStream = new DataFileStream<>(inStream, elementReader);
+		for (int i = 0; i < numElements; i++) {
+			AvroKeyValueSinkWriter.AvroKeyValue<Integer, String> wrappedEntry =
+				new AvroKeyValueSinkWriter.AvroKeyValue<>(dataFileStream.next());
+			int key = wrappedEntry.getKey();
+			Assert.assertEquals(i, key);
+			String value = wrappedEntry.getValue();
+			Assert.assertEquals("message #" + i, value);
+		}
+
+		dataFileStream.close();
+		inStream.close();
+	}
+
+	/**
+	 * This uses {@link DateTimeBucketer} to
+	 * produce rolling files. We use {@link OneInputStreamOperatorTestHarness} to manually
+	 * advance processing time.
+	 */
+	@Test
+	public void testDateTimeRollingStringWriter() throws Exception {
+		final int numElements = 20;
+
+		final String outPath = hdfsURI + "/rolling-out";
+
+		BucketingSink<String> sink = new BucketingSink<String>(outPath)
+			.setBucketer(new DateTimeBucketer<String>("ss"))
+			.setPartPrefix(PART_PREFIX)
+			.setPendingPrefix("")
+			.setPendingSuffix("");
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(sink, 1, 0);
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.setup();
+		testHarness.open();
+
+		for (int i = 0; i < numElements; i++) {
+			// Every 5 elements, increase the clock time. We should end up with 5 elements per bucket.
+			if (i % 5 == 0) {
+				testHarness.setProcessingTime(i * 1000L);
+			}
+			testHarness.processElement(new StreamRecord<>("message #" + Integer.toString(i)));
+		}
+
+		testHarness.close();
+
+		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(outPath), true);
+
+		// We should have 4 rolling files across 4 time intervals
+		int numFiles = 0;
+		while (files.hasNext()) {
+			LocatedFileStatus file = files.next();
+			numFiles++;
+			if (file.getPath().toString().contains("rolling-out/00")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 0; i < 5; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else if (file.getPath().toString().contains("rolling-out/05")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 5; i < 10; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else if (file.getPath().toString().contains("rolling-out/10")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 10; i < 15; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else if (file.getPath().toString().contains("rolling-out/15")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 15; i < 20; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else {
+				Assert.fail("File " + file + " does not match any expected roll pattern.");
+			}
+		}
+
+		Assert.assertEquals(4, numFiles);
+	}
+
+	/**
+	 * This uses a custom bucketing function which determines the bucket from the input.
+	 */
+	@Test
+	public void testCustomBucketing() throws Exception {
+		File dataDir = tempFolder.newFolder();
+
+		final int numIds = 4;
+		final int numElements = 20;
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(dataDir, 1, 0);
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.setup();
+		testHarness.open();
+
+		for (int i = 0; i < numElements; i++) {
+			testHarness.processElement(new StreamRecord<>(Integer.toString(i % numIds)));
+		}
+
+		testHarness.close();
+
+		// we should have 4 buckets, with 1 file each
+		int numFiles = 0;
+		for (File file: FileUtils.listFiles(dataDir, null, true)) {
+			if (file.getName().startsWith(PART_PREFIX)) {
+				numFiles++;
+			}
+		}
+
+		Assert.assertEquals(4, numFiles);
+	}
+
+	/**
+	 * This uses a custom bucketing function which determines the bucket from the input.
+	 * We use a simulated clock to reduce the number of buckets being written to over time.
+	 * This causes buckets to become 'inactive' and their file parts 'closed' by the sink.
+	 */
+	@Test
+	public void testCustomBucketingInactiveBucketCleanup() throws Exception {
+		File dataDir = tempFolder.newFolder();
+
+		final int step1NumIds = 4;
+		final int step2NumIds = 2;
+		final int numElementsPerStep = 20;
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(dataDir, 1, 0);
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.setup();
+		testHarness.open();
+
+		for (int i = 0; i < numElementsPerStep; i++) {
+			testHarness.processElement(new StreamRecord<>(Integer.toString(i % step1NumIds)));
+		}
+
+		testHarness.setProcessingTime(2*60*1000L);
+
+		for (int i = 0; i < numElementsPerStep; i++) {
+			testHarness.processElement(new StreamRecord<>(Integer.toString(i % step2NumIds)));
+		}
+
+		testHarness.setProcessingTime(6*60*1000L);
+
+		for (int i = 0; i < numElementsPerStep; i++) {
+			testHarness.processElement(new StreamRecord<>(Integer.toString(i % step2NumIds)));
+		}
+
+		// we should have 4 buckets, with 1 file each
+		// 2 of these buckets should have been finalised due to becoming inactive
+		int numFiles = 0;
+		int numInProgress = 0;
+		for (File file: FileUtils.listFiles(dataDir, null, true)) {
+			if (file.getAbsolutePath().endsWith("crc")) {
+				continue;
+			}
+			if (file.getPath().endsWith(IN_PROGRESS_SUFFIX)) {
+				numInProgress++;
+			}
+			numFiles++;
+		}
+
+		testHarness.close();
+
+		Assert.assertEquals(4, numFiles);
+		Assert.assertEquals(2, numInProgress);
+	}
+
+	/**
+	 * This tests user defined hdfs configuration
+	 * @throws Exception
+	 */
+	@Test
+	public void testUserDefinedConfiguration() throws Exception {
+		final String outPath = hdfsURI + "/string-non-rolling-with-config";
+		final int numElements = 20;
+
+		Map<String, String> properties = new HashMap<>();
+		Schema keySchema = Schema.create(Schema.Type.INT);
+		Schema valueSchema = Schema.create(Schema.Type.STRING);
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema.toString());
+		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema.toString());
+		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS, String.valueOf(true));
+		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS_CODEC, DataFileConstants.SNAPPY_CODEC);
+
+		Configuration conf = new Configuration();
+		conf.set("io.file.buffer.size", "40960");
+
+		BucketingSink<Tuple2<Integer,String>> sink = new BucketingSink<Tuple2<Integer, String>>(outPath)
+			.setFSConfig(conf)
+			.setWriter(new StreamWriterWithConfigCheck<Integer, String>(properties, "io.file.buffer.size", "40960"))
+			.setBucketer(new BasePathBucketer<Tuple2<Integer,String>>())
+			.setPartPrefix(PART_PREFIX)
+			.setPendingPrefix("")
+			.setPendingSuffix("");
+
+		OneInputStreamOperatorTestHarness<Tuple2<Integer, String>, Object> testHarness =
+			createTestSink(sink, 1, 0);
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.setup();
+		testHarness.open();
+
+		for (int i = 0; i < numElements; i++) {
+			testHarness.processElement(new StreamRecord<>(Tuple2.of(
+				i, "message #" + Integer.toString(i)
+			)));
+		}
+
+		testHarness.close();
+
+		GenericData.setStringType(valueSchema, GenericData.StringType.String);
+		Schema elementSchema = AvroKeyValueSinkWriter.AvroKeyValue.getSchema(keySchema, valueSchema);
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/" + PART_PREFIX + "-0-0"));
+
+		SpecificDatumReader<GenericRecord> elementReader = new SpecificDatumReader<>(elementSchema);
+		DataFileStream<GenericRecord> dataFileStream = new DataFileStream<>(inStream, elementReader);
+		for (int i = 0; i < numElements; i++) {
+			AvroKeyValueSinkWriter.AvroKeyValue<Integer, String> wrappedEntry =
+				new AvroKeyValueSinkWriter.AvroKeyValue<>(dataFileStream.next());
+			int key = wrappedEntry.getKey();
+			Assert.assertEquals(i, key);
+			String value = wrappedEntry.getValue();
+			Assert.assertEquals("message #" + i, value);
+		}
+
+		dataFileStream.close();
+		inStream.close();
+	}
+
+	private static class StreamWriterWithConfigCheck<K, V> extends AvroKeyValueSinkWriter<K, V> {
+		private Map<String, String> properties;
+		private String key;
+		private String expect;
+		public StreamWriterWithConfigCheck(Map<String, String> properties, String key, String expect) {
+			super(properties);
+			this.properties = properties;
+			this.key = key;
+			this.expect = expect;
+		}
+
+		@Override
+		public void open(FileSystem fs, Path path) throws IOException {
+			super.open(fs, path);
+			Assert.assertEquals(expect, fs.getConf().get(key));
+		}
+
+		@Override
+		public Writer<Tuple2<K, V>> duplicate() {
+			return new StreamWriterWithConfigCheck<>(properties, key, expect);
+		}
+	}
+
+}

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

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


[15/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
deleted file mode 100644
index cf2c373..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
+++ /dev/null
@@ -1,1082 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs.bucketing;
-
-import org.apache.commons.lang3.time.StopWatch;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
-import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.fs.Clock;
-import org.apache.flink.streaming.connectors.fs.SequenceFileWriter;
-import org.apache.flink.streaming.connectors.fs.StringWriter;
-import org.apache.flink.streaming.connectors.fs.Writer;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.util.Preconditions;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.Iterator;
-
-/**
- * Sink that emits its input elements to {@link org.apache.hadoop.fs.FileSystem} files within
- * buckets. This is integrated with the checkpointing mechanism to provide exactly once semantics.
- *
- * <p>
- * When creating the sink a {@code basePath} must be specified. The base directory contains
- * one directory for every bucket. The bucket directories themselves contain several part files,
- * one for each parallel subtask of the sink. These part files contain the actual output data.
- *
- * <p>
- * The sink uses a {@link Bucketer} to determine in which bucket directory each element should
- * be written to inside the base directory. The {@code Bucketer} can, for example, use time or
- * a property of the element to determine the bucket directory. The default {@code Bucketer} is a
- * {@link DateTimeBucketer} which will create one new bucket every hour. You can specify
- * a custom {@code Bucketer} using {@link #setBucketer(Bucketer)}. For example, use the
- * {@link BasePathBucketer} if you don't want to have buckets but still want to write part-files
- * in a fault-tolerant way.
- *
- * <p>
- * The filenames of the part files contain the part prefix, the parallel subtask index of the sink
- * and a rolling counter. For example the file {@code "part-1-17"} contains the data from
- * {@code subtask 1} of the sink and is the {@code 17th} bucket created by that subtask. Per default
- * the part prefix is {@code "part"} but this can be configured using {@link #setPartPrefix(String)}.
- * When a part file becomes bigger than the user-specified batch size the current part file is closed,
- * the part counter is increased and a new part file is created. The batch size defaults to {@code 384MB},
- * this can be configured using {@link #setBatchSize(long)}.
- *
- * <p>
- * In some scenarios, the open buckets are required to change based on time. In these cases, the sink
- * needs to determine when a bucket has become inactive, in order to flush and close the part file.
- * To support this there are two configurable settings:
- * <ol>
- *     <li>the frequency to check for inactive buckets, configured by {@link #setInactiveBucketCheckInterval(long)},
- *     and</li>
- *     <li>the minimum amount of time a bucket has to not receive any data before it is considered inactive,
- *     configured by {@link #setInactiveBucketThreshold(long)}</li>
- * </ol>
- * Both of these parameters default to {@code 60,000 ms}, or {@code 1 min}.
- *
- * <p>
- * Part files can be in one of three states: {@code in-progress}, {@code pending} or {@code finished}.
- * The reason for this is how the sink works together with the checkpointing mechanism to provide exactly-once
- * semantics and fault-tolerance. The part file that is currently being written to is {@code in-progress}. Once
- * a part file is closed for writing it becomes {@code pending}. When a checkpoint is successful the currently
- * pending files will be moved to {@code finished}.
- *
- * <p>
- * If case of a failure, and in order to guarantee exactly-once semantics, the sink should roll back to the state it
- * had when that last successful checkpoint occurred. To this end, when restoring, the restored files in {@code pending}
- * state are transferred into the {@code finished} state while any {@code in-progress} files are rolled back, so that
- * they do not contain data that arrived after the checkpoint from which we restore. If the {@code FileSystem} supports
- * the {@code truncate()} method this will be used to reset the file back to its previous state. If not, a special
- * file with the same name as the part file and the suffix {@code ".valid-length"} will be created that contains the
- * length up to which the file contains valid data. When reading the file, it must be ensured that it is only read up
- * to that point. The prefixes and suffixes for the different file states and valid-length files can be configured
- * using the adequate setter method, e.g. {@link #setPendingSuffix(String)}.
- *
- * <p>
- * <b>NOTE:</b>
- * <ol>
- *     <li>
- *         If checkpointing is not enabled the pending files will never be moved to the finished state. In that case,
- *         the pending suffix/prefix can be set to {@code ""} to make the sink work in a non-fault-tolerant way but
- *         still provide output without prefixes and suffixes.
- *     </li>
- *     <li>
- *         The part files are written using an instance of {@link Writer}. By default, a
- *         {@link org.apache.flink.streaming.connectors.fs.StringWriter} is used, which writes the result
- *         of {@code toString()} for every element, separated by newlines. You can configure the writer using  the
- *         {@link #setWriter(Writer)}. For example, {@link org.apache.flink.streaming.connectors.fs.SequenceFileWriter}
- *         can be used to write Hadoop {@code SequenceFiles}.
- *     </li>
- * </ol>
- *
- * <p>
- * Example:
- * <pre>{@code
- *     new BucketingSink<Tuple2<IntWritable, Text>>(outPath)
- *         .setWriter(new SequenceFileWriter<IntWritable, Text>())
- *         .setBucketer(new DateTimeBucketer("yyyy-MM-dd--HHmm")
- * }</pre>
- *
- * This will create a sink that writes to {@code SequenceFiles} and rolls every minute.
- *
- * @see DateTimeBucketer
- * @see StringWriter
- * @see SequenceFileWriter
- *
- * @param <T> Type of the elements emitted by this sink
- */
-public class BucketingSink<T>
-		extends RichSinkFunction<T>
-		implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener, ProcessingTimeCallback {
-
-	private static final long serialVersionUID = 1L;
-
-	private static Logger LOG = LoggerFactory.getLogger(BucketingSink.class);
-
-	// --------------------------------------------------------------------------------------------
-	//  User configuration values
-	// --------------------------------------------------------------------------------------------
-	// These are initialized with some defaults but are meant to be changeable by the user
-
-	/**
-	 * The default maximum size of part files (currently {@code 384 MB}).
-	 */
-	private final long DEFAULT_BATCH_SIZE = 1024L * 1024L * 384L;
-
-	/**
-	 * The default time between checks for inactive buckets. By default, {60 sec}.
-	 */
-	private final long DEFAULT_INACTIVE_BUCKET_CHECK_INTERVAL_MS = 60 * 1000L;
-
-	/**
-	 * The default threshold (in {@code ms}) for marking a bucket as inactive and
-	 * closing its part files. By default, {60 sec}.
-	 */
-	private final long DEFAULT_INACTIVE_BUCKET_THRESHOLD_MS = 60 * 1000L;
-
-	/**
-	 * The suffix for {@code in-progress} part files. These are files we are
-	 * currently writing to, but which were not yet confirmed by a checkpoint.
-	 */
-	private final String DEFAULT_IN_PROGRESS_SUFFIX = ".in-progress";
-
-	/**
-	 * The prefix for {@code in-progress} part files. These are files we are
-	 * currently writing to, but which were not yet confirmed by a checkpoint.
-	 */
-	private final String DEFAULT_IN_PROGRESS_PREFIX = "_";
-
-	/**
-	 * The suffix for {@code pending} part files. These are closed files that we are
-	 * not currently writing to (inactive or reached {@link #batchSize}), but which
-	 * were not yet confirmed by a checkpoint.
-	 */
-	private final String DEFAULT_PENDING_SUFFIX = ".pending";
-
-	/**
-	 * The prefix for {@code pending} part files. These are closed files that we are
-	 * not currently writing to (inactive or reached {@link #batchSize}), but which
-	 * were not yet confirmed by a checkpoint.
-	 */
-	private final String DEFAULT_PENDING_PREFIX = "_";
-
-	/**
-	 * When {@code truncate()} is not supported by the used {@link FileSystem}, we create
-	 * a file along the part file with this suffix that contains the length up to which
-	 * the part file is valid.
-	 */
-	private final String DEFAULT_VALID_SUFFIX = ".valid-length";
-
-	/**
-	 * When {@code truncate()} is not supported by the used {@link FileSystem}, we create
-	 * a file along the part file with this preffix that contains the length up to which
-	 * the part file is valid.
-	 */
-	private final String DEFAULT_VALID_PREFIX = "_";
-
-	/**
-	 * The default prefix for part files.
-	 */
-	private final String DEFAULT_PART_REFIX = "part";
-
-	/**
-	 * The default timeout for asynchronous operations such as recoverLease and truncate (in {@code ms}).
-	 */
-	private final long DEFAULT_ASYNC_TIMEOUT_MS = 60 * 1000;
-
-
-	/**
-	 * The base {@code Path} that stores all bucket directories.
-	 */
-	private final String basePath;
-
-	/**
-	 * The {@code Bucketer} that is used to determine the path of bucket directories.
-	 */
-	private Bucketer<T> bucketer;
-
-	/**
-	 * We have a template and call duplicate() for each parallel writer in open() to get the actual
-	 * writer that is used for the part files.
-	 */
-	private Writer<T> writerTemplate;
-
-	private long batchSize = DEFAULT_BATCH_SIZE;
-	private long inactiveBucketCheckInterval = DEFAULT_INACTIVE_BUCKET_CHECK_INTERVAL_MS;
-	private long inactiveBucketThreshold = DEFAULT_INACTIVE_BUCKET_THRESHOLD_MS;
-
-	// These are the actually configured prefixes/suffixes
-	private String inProgressSuffix = DEFAULT_IN_PROGRESS_SUFFIX;
-	private String inProgressPrefix = DEFAULT_IN_PROGRESS_PREFIX;
-
-	private String pendingSuffix = DEFAULT_PENDING_SUFFIX;
-	private String pendingPrefix = DEFAULT_PENDING_PREFIX;
-
-	private String validLengthSuffix = DEFAULT_VALID_SUFFIX;
-	private String validLengthPrefix= DEFAULT_VALID_PREFIX;
-
-	private String partPrefix = DEFAULT_PART_REFIX;
-
-	/**
-	 * The timeout for asynchronous operations such as recoverLease and truncate (in {@code ms}).
-	 */
-	private long asyncTimeout = DEFAULT_ASYNC_TIMEOUT_MS;
-
-	// --------------------------------------------------------------------------------------------
-	//  Internal fields (not configurable by user)
-	// -------------------------------------------�-------------------------------------------------
-
-	/**
-	 * We use reflection to get the .truncate() method, this is only available starting with Hadoop 2.7
-	 */
-	private transient Method refTruncate;
-
-	/**
-	 * The state object that is handled by Flink from snapshot/restore. This contains state for
-	 * every open bucket: the current in-progress part file path, its valid length and the pending part files.
-	 */
-	private transient State<T> state;
-
-	private transient ListState<State<T>> restoredBucketStates;
-
-	/**
-	 * User-defined FileSystem parameters
-	 */
-	private Configuration fsConfig;
-
-	/**
-	 * The FileSystem reference.
-	 */
-	private transient FileSystem fs;
-
-	private transient Clock clock;
-
-	private transient ProcessingTimeService processingTimeService;
-
-	/**
-	 * Creates a new {@code BucketingSink} that writes files to the given base directory.
-	 *
-	 * <p>
-	 * This uses a{@link DateTimeBucketer} as {@link Bucketer} and a {@link StringWriter} has writer.
-	 * The maximum bucket size is set to 384 MB.
-	 *
-	 * @param basePath The directory to which to write the bucket files.
-	 */
-	public BucketingSink(String basePath) {
-		this.basePath = basePath;
-		this.bucketer = new DateTimeBucketer<>();
-		this.writerTemplate = new StringWriter<>();
-	}
-
-	/**
-	 * Specify a custom {@code Configuration} that will be used when creating
-	 * the {@link FileSystem} for writing.
-	 */
-	public BucketingSink<T> setFSConfig(Configuration config) {
-		this.fsConfig = new Configuration();
-		fsConfig.addAll(config);
-		return this;
-	}
-
-	/**
-	 * Specify a custom {@code Configuration} that will be used when creating
-	 * the {@link FileSystem} for writing.
-	 */
-	public BucketingSink<T> setFSConfig(org.apache.hadoop.conf.Configuration config) {
-		this.fsConfig = new Configuration();
-		for(Map.Entry<String, String> entry : config) {
-			fsConfig.setString(entry.getKey(), entry.getValue());
-		}
-		return this;
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
-		if (this.writerTemplate instanceof InputTypeConfigurable) {
-			((InputTypeConfigurable) writerTemplate).setInputType(type, executionConfig);
-		}
-	}
-
-	@Override
-	public void initializeState(FunctionInitializationContext context) throws Exception {
-		Preconditions.checkArgument(this.restoredBucketStates == null, "The operator has already been initialized.");
-
-		initFileSystem();
-
-		if (this.refTruncate == null) {
-			this.refTruncate = reflectTruncate(fs);
-		}
-
-		OperatorStateStore stateStore = context.getOperatorStateStore();
-		restoredBucketStates = stateStore.getSerializableListState("bucket-states");
-
-		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
-		if (context.isRestored()) {
-			LOG.info("Restoring state for the {} (taskIdx={}).", getClass().getSimpleName(), subtaskIndex);
-
-			for (State<T> recoveredState : restoredBucketStates.get()) {
-				handleRestoredBucketState(recoveredState);
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("{} idx {} restored {}", getClass().getSimpleName(), subtaskIndex, recoveredState);
-				}
-			}
-		} else {
-			LOG.info("No state to restore for the {} (taskIdx={}).", getClass().getSimpleName(), subtaskIndex);
-		}
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-
-		state = new State<>();
-
-		processingTimeService =
-				((StreamingRuntimeContext) getRuntimeContext()).getProcessingTimeService();
-
-		long currentProcessingTime = processingTimeService.getCurrentProcessingTime();
-
-		processingTimeService.registerTimer(currentProcessingTime + inactiveBucketCheckInterval, this);
-
-		this.clock = new Clock() {
-			@Override
-			public long currentTimeMillis() {
-				return processingTimeService.getCurrentProcessingTime();
-			}
-		};
-	}
-
-	/**
-	 * Create a file system with the user-defined {@code HDFS} configuration.
-	 * @throws IOException
-	 */
-	private void initFileSystem() throws IOException {
-		if (fs != null) {
-			return;
-		}
-		org.apache.hadoop.conf.Configuration hadoopConf = HadoopFileSystem.getHadoopConfiguration();
-		if (fsConfig != null) {
-			String disableCacheName = String.format("fs.%s.impl.disable.cache", new Path(basePath).toUri().getScheme());
-			hadoopConf.setBoolean(disableCacheName, true);
-			for (String key : fsConfig.keySet()) {
-				hadoopConf.set(key, fsConfig.getString(key, null));
-			}
-		}
-
-		fs = new Path(basePath).getFileSystem(hadoopConf);
-	}
-
-	@Override
-	public void close() throws Exception {
-		for (Map.Entry<String, BucketState<T>> entry : state.bucketStates.entrySet()) {
-			closeCurrentPartFile(entry.getValue());
-		}
-	}
-
-	@Override
-	public void invoke(T value) throws Exception {
-		Path bucketPath = bucketer.getBucketPath(clock, new Path(basePath), value);
-
-		long currentProcessingTime = processingTimeService.getCurrentProcessingTime();
-
-		BucketState<T> bucketState = state.getBucketState(bucketPath);
-		if (bucketState == null) {
-			bucketState = new BucketState<>(currentProcessingTime);
-			state.addBucketState(bucketPath, bucketState);
-		}
-
-		if (shouldRoll(bucketState)) {
-			openNewPartFile(bucketPath, bucketState);
-		}
-
-		bucketState.writer.write(value);
-		bucketState.lastWrittenToTime = currentProcessingTime;
-	}
-
-	/**
-	 * Returns {@code true} if the current {@code part-file} should be closed and a new should be created.
-	 * This happens if:
-	 * <ol>
-	 *     <li>no file is created yet for the task to write to, or</li>
-	 *     <li>the current file has reached the maximum bucket size.</li>
-	 * </ol>
-	 */
-	private boolean shouldRoll(BucketState<T> bucketState) throws IOException {
-		boolean shouldRoll = false;
-		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
-		if (!bucketState.isWriterOpen) {
-			shouldRoll = true;
-			LOG.debug("BucketingSink {} starting new bucket.", subtaskIndex);
-		} else {
-			long writePosition = bucketState.writer.getPos();
-			if (writePosition > batchSize) {
-				shouldRoll = true;
-				LOG.debug(
-					"BucketingSink {} starting new bucket because file position {} is above batch size {}.",
-					subtaskIndex,
-					writePosition,
-					batchSize);
-			}
-		}
-		return shouldRoll;
-	}
-
-	@Override
-	public void onProcessingTime(long timestamp) throws Exception {
-		long currentProcessingTime = processingTimeService.getCurrentProcessingTime();
-
-		checkForInactiveBuckets(currentProcessingTime);
-
-		processingTimeService.registerTimer(currentProcessingTime + inactiveBucketCheckInterval, this);
-	}
-
-	/**
-	 * Checks for inactive buckets, and closes them. Buckets are considered inactive if they have not been
-	 * written to for a period greater than {@code inactiveBucketThreshold} ms. This enables in-progress
-	 * files to be moved to the pending state and be finalised on the next checkpoint.
-	 */
-	private void checkForInactiveBuckets(long currentProcessingTime) throws Exception {
-
-		synchronized (state.bucketStates) {
-			for (Map.Entry<String, BucketState<T>> entry : state.bucketStates.entrySet()) {
-				if (entry.getValue().lastWrittenToTime < currentProcessingTime - inactiveBucketThreshold) {
-					LOG.debug("BucketingSink {} closing bucket due to inactivity of over {} ms.",
-						getRuntimeContext().getIndexOfThisSubtask(), inactiveBucketThreshold);
-					closeCurrentPartFile(entry.getValue());
-				}
-			}
-		}
-	}
-
-	/**
-	 * Closes the current part file and opens a new one with a new bucket path, as returned by the
-	 * {@link Bucketer}. If the bucket is not new, then this will create a new file with the same path
-	 * as its predecessor, but with an increased rolling counter (see {@link BucketingSink}.
-	 */
-	private void openNewPartFile(Path bucketPath, BucketState<T> bucketState) throws Exception {
-		closeCurrentPartFile(bucketState);
-
-		if (!fs.exists(bucketPath)) {
-			try {
-				if (fs.mkdirs(bucketPath)) {
-					LOG.debug("Created new bucket directory: {}", bucketPath);
-				}
-			} catch (IOException e) {
-				throw new RuntimeException("Could not create new bucket path.", e);
-			}
-		}
-
-		// The following loop tries different partCounter values in ascending order until it reaches the minimum
-		// that is not yet used. This works since there is only one parallel subtask that tries names with this
-		// subtask id. Otherwise we would run into concurrency issues here. This is aligned with the way we now
-		// clean the base directory in case of rescaling.
-
-		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
-		Path partPath = new Path(bucketPath, partPrefix + "-" + subtaskIndex + "-" + bucketState.partCounter);
-		while (fs.exists(partPath) ||
-				fs.exists(getPendingPathFor(partPath)) ||
-				fs.exists(getInProgressPathFor(partPath))) {
-			bucketState.partCounter++;
-			partPath = new Path(bucketPath, partPrefix + "-" + subtaskIndex + "-" + bucketState.partCounter);
-		}
-
-		// increase, so we don't have to check for this name next time
-		bucketState.partCounter++;
-
-		LOG.debug("Next part path is {}", partPath.toString());
-		bucketState.currentFile = partPath.toString();
-
-		Path inProgressPath = getInProgressPathFor(partPath);
-		if (bucketState.writer == null) {
-			bucketState.writer = writerTemplate.duplicate();
-		}
-
-		bucketState.writer.open(fs, inProgressPath);
-		bucketState.isWriterOpen = true;
-	}
-
-	/**
-	 * Closes the current part file and moves it from the in-progress state to the pending state.
-	 */
-	private void closeCurrentPartFile(BucketState<T> bucketState) throws Exception {
-		if (bucketState.isWriterOpen) {
-			bucketState.writer.close();
-			bucketState.isWriterOpen = false;
-		}
-
-		if (bucketState.currentFile != null) {
-			Path currentPartPath = new Path(bucketState.currentFile);
-			Path inProgressPath = getInProgressPathFor(currentPartPath);
-			Path pendingPath = getPendingPathFor(currentPartPath);
-
-			fs.rename(inProgressPath, pendingPath);
-			LOG.debug("Moving in-progress bucket {} to pending file {}",
-				inProgressPath,
-				pendingPath);
-			bucketState.pendingFiles.add(currentPartPath.toString());
-			bucketState.currentFile = null;
-		}
-	}
-
-	/**
-	 * Gets the truncate() call using reflection.
-	 * <p>
-	 * <b>NOTE:</b> This code comes from Flume.
-	 */
-	private Method reflectTruncate(FileSystem fs) {
-		Method m = null;
-		if(fs != null) {
-			Class<?> fsClass = fs.getClass();
-			try {
-				m = fsClass.getMethod("truncate", Path.class, long.class);
-			} catch (NoSuchMethodException ex) {
-				LOG.debug("Truncate not found. Will write a file with suffix '{}' " +
-					" and prefix '{}' to specify how many bytes in a bucket are valid.", validLengthSuffix, validLengthPrefix);
-				return null;
-			}
-
-			// verify that truncate actually works
-			FSDataOutputStream outputStream;
-			Path testPath = new Path(UUID.randomUUID().toString());
-			try {
-				outputStream = fs.create(testPath);
-				outputStream.writeUTF("hello");
-				outputStream.close();
-			} catch (IOException e) {
-				LOG.error("Could not create file for checking if truncate works.", e);
-				throw new RuntimeException("Could not create file for checking if truncate works.", e);
-			}
-
-			try {
-				m.invoke(fs, testPath, 2);
-			} catch (IllegalAccessException | InvocationTargetException e) {
-				LOG.debug("Truncate is not supported.", e);
-				m = null;
-			}
-
-			try {
-				fs.delete(testPath, false);
-			} catch (IOException e) {
-				LOG.error("Could not delete truncate test file.", e);
-				throw new RuntimeException("Could not delete truncate test file.", e);
-			}
-		}
-		return m;
-	}
-
-	private Path getPendingPathFor(Path path) {
-		return new Path(path.getParent(), pendingPrefix + path.getName()).suffix(pendingSuffix);
-	}
-
-	private Path getInProgressPathFor(Path path) {
-		return new Path(path.getParent(), inProgressPrefix + path.getName()).suffix(inProgressSuffix);
-	}
-
-	private Path getValidLengthPathFor(Path path) {
-		return new Path(path.getParent(), validLengthPrefix + path.getName()).suffix(validLengthSuffix);
-	}
-
-	@Override
-	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		synchronized (state.bucketStates) {
-
-			Iterator<Map.Entry<String, BucketState<T>>> bucketStatesIt = state.bucketStates.entrySet().iterator();
-			while (bucketStatesIt.hasNext()) {
-				BucketState<T> bucketState = bucketStatesIt.next().getValue();
-				synchronized (bucketState.pendingFilesPerCheckpoint) {
-
-					Iterator<Map.Entry<Long, List<String>>> pendingCheckpointsIt =
-						bucketState.pendingFilesPerCheckpoint.entrySet().iterator();
-
-					while (pendingCheckpointsIt.hasNext()) {
-
-						Map.Entry<Long, List<String>> entry = pendingCheckpointsIt.next();
-						Long pastCheckpointId = entry.getKey();
-						List<String> pendingPaths = entry.getValue();
-
-						if (pastCheckpointId <= checkpointId) {
-							LOG.debug("Moving pending files to final location for checkpoint {}", pastCheckpointId);
-
-							for (String filename : pendingPaths) {
-								Path finalPath = new Path(filename);
-								Path pendingPath = getPendingPathFor(finalPath);
-
-								fs.rename(pendingPath, finalPath);
-								LOG.debug(
-									"Moving pending file {} to final location having completed checkpoint {}.",
-									pendingPath,
-									pastCheckpointId);
-							}
-							pendingCheckpointsIt.remove();
-						}
-					}
-
-					if (!bucketState.isWriterOpen &&
-						bucketState.pendingFiles.isEmpty() &&
-						bucketState.pendingFilesPerCheckpoint.isEmpty()) {
-
-						// We've dealt with all the pending files and the writer for this bucket is not currently open.
-						// Therefore this bucket is currently inactive and we can remove it from our state.
-						bucketStatesIt.remove();
-					}
-				}
-			}
-		}
-	}
-
-	@Override
-	public void snapshotState(FunctionSnapshotContext context) throws Exception {
-		Preconditions.checkNotNull(restoredBucketStates, "The operator has not been properly initialized.");
-
-		restoredBucketStates.clear();
-
-		synchronized (state.bucketStates) {
-			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
-
-			for (Map.Entry<String, BucketState<T>> bucketStateEntry : state.bucketStates.entrySet()) {
-				BucketState<T> bucketState = bucketStateEntry.getValue();
-
-				if (bucketState.isWriterOpen) {
-					bucketState.currentFileValidLength = bucketState.writer.flush();
-				}
-
-				synchronized (bucketState.pendingFilesPerCheckpoint) {
-					bucketState.pendingFilesPerCheckpoint.put(context.getCheckpointId(), bucketState.pendingFiles);
-				}
-				bucketState.pendingFiles = new ArrayList<>();
-			}
-			restoredBucketStates.add(state);
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("{} idx {} checkpointed {}.", getClass().getSimpleName(), subtaskIdx, state);
-			}
-		}
-	}
-
-	private void handleRestoredBucketState(State<T> restoredState) {
-		Preconditions.checkNotNull(restoredState);
-
-		for (BucketState<T> bucketState : restoredState.bucketStates.values()) {
-
-			// we can clean all the pending files since they were renamed to
-			// final files after this checkpoint was successful
-			// (we re-start from the last **successful** checkpoint)
-			bucketState.pendingFiles.clear();
-
-			if (bucketState.currentFile != null) {
-
-				// We were writing to a file when the last checkpoint occurred. This file can either
-				// be still in-progress or became a pending file at some point after the checkpoint.
-				// Either way, we have to truncate it back to a valid state (or write a .valid-length
-				// file that specifies up to which length it is valid) and rename it to the final name
-				// before starting a new bucket file.
-
-				Path partPath = new Path(bucketState.currentFile);
-				try {
-					Path partPendingPath = getPendingPathFor(partPath);
-					Path partInProgressPath = getInProgressPathFor(partPath);
-
-					if (fs.exists(partPendingPath)) {
-						LOG.debug("In-progress file {} has been moved to pending after checkpoint, moving to final location.", partPath);
-						// has been moved to pending in the mean time, rename to final location
-						fs.rename(partPendingPath, partPath);
-					} else if (fs.exists(partInProgressPath)) {
-						LOG.debug("In-progress file {} is still in-progress, moving to final location.", partPath);
-						// it was still in progress, rename to final path
-						fs.rename(partInProgressPath, partPath);
-					} else if (fs.exists(partPath)) {
-						LOG.debug("In-Progress file {} was already moved to final location {}.", bucketState.currentFile, partPath);
-					} else {
-						LOG.debug("In-Progress file {} was neither moved to pending nor is still in progress. Possibly, " +
-							"it was moved to final location by a previous snapshot restore", bucketState.currentFile);
-					}
-
-					// We use reflection to get the .truncate() method, this
-					// is only available starting with Hadoop 2.7
-					if (this.refTruncate == null) {
-						this.refTruncate = reflectTruncate(fs);
-					}
-
-					// truncate it or write a ".valid-length" file to specify up to which point it is valid
-					if (refTruncate != null) {
-						LOG.debug("Truncating {} to valid length {}", partPath, bucketState.currentFileValidLength);
-						// some-one else might still hold the lease from a previous try, we are
-						// recovering, after all ...
-						if (fs instanceof DistributedFileSystem) {
-							DistributedFileSystem dfs = (DistributedFileSystem) fs;
-							LOG.debug("Trying to recover file lease {}", partPath);
-							dfs.recoverLease(partPath);
-							boolean isclosed = dfs.isFileClosed(partPath);
-							StopWatch sw = new StopWatch();
-							sw.start();
-							while (!isclosed) {
-								if (sw.getTime() > asyncTimeout) {
-									break;
-								}
-								try {
-									Thread.sleep(500);
-								} catch (InterruptedException e1) {
-									// ignore it
-								}
-								isclosed = dfs.isFileClosed(partPath);
-							}
-						}
-						Boolean truncated = (Boolean) refTruncate.invoke(fs, partPath, bucketState.currentFileValidLength);
-						if (!truncated) {
-							LOG.debug("Truncate did not immediately complete for {}, waiting...", partPath);
-
-							// we must wait for the asynchronous truncate operation to complete
-							StopWatch sw = new StopWatch();
-							sw.start();
-							long newLen = fs.getFileStatus(partPath).getLen();
-							while (newLen != bucketState.currentFileValidLength) {
-								if (sw.getTime() > asyncTimeout) {
-									break;
-								}
-								try {
-									Thread.sleep(500);
-								} catch (InterruptedException e1) {
-									// ignore it
-								}
-								newLen = fs.getFileStatus(partPath).getLen();
-							}
-							if (newLen != bucketState.currentFileValidLength) {
-								throw new RuntimeException("Truncate did not truncate to right length. Should be " + bucketState.currentFileValidLength + " is " + newLen + ".");
-							}
-						}
-					} else {
-						LOG.debug("Writing valid-length file for {} to specify valid length {}", partPath, bucketState.currentFileValidLength);
-						Path validLengthFilePath = getValidLengthPathFor(partPath);
-						if (!fs.exists(validLengthFilePath)) {
-							FSDataOutputStream lengthFileOut = fs.create(validLengthFilePath);
-							lengthFileOut.writeUTF(Long.toString(bucketState.currentFileValidLength));
-							lengthFileOut.close();
-						}
-					}
-
-					// Now that we've restored the bucket to a valid state, reset the current file info
-					bucketState.currentFile = null;
-					bucketState.currentFileValidLength = -1;
-					bucketState.isWriterOpen = false;
-				} catch (IOException e) {
-					LOG.error("Error while restoring BucketingSink state.", e);
-					throw new RuntimeException("Error while restoring BucketingSink state.", e);
-				} catch (InvocationTargetException | IllegalAccessException e) {
-					LOG.error("Could not invoke truncate.", e);
-					throw new RuntimeException("Could not invoke truncate.", e);
-				}
-			}
-
-			// Move files that are confirmed by a checkpoint but did not get moved to final location
-			// because the checkpoint notification did not happen before a failure
-
-			LOG.debug("Moving pending files to final location on restore.");
-
-			Set<Long> pastCheckpointIds = bucketState.pendingFilesPerCheckpoint.keySet();
-			for (Long pastCheckpointId : pastCheckpointIds) {
-				// All the pending files are buckets that have been completed but are waiting to be renamed
-				// to their final name
-				for (String filename : bucketState.pendingFilesPerCheckpoint.get(pastCheckpointId)) {
-					Path finalPath = new Path(filename);
-					Path pendingPath = getPendingPathFor(finalPath);
-
-					try {
-						if (fs.exists(pendingPath)) {
-							LOG.debug("Restoring BucketingSink State: Moving pending file {} to final location after complete checkpoint {}.", pendingPath, pastCheckpointId);
-							fs.rename(pendingPath, finalPath);
-						}
-					} catch (IOException e) {
-						LOG.error("Restoring BucketingSink State: Error while renaming pending file {} to final path {}: {}", pendingPath, finalPath, e);
-						throw new RuntimeException("Error while renaming pending file " + pendingPath + " to final path " + finalPath, e);
-					}
-				}
-			}
-
-			synchronized (bucketState.pendingFilesPerCheckpoint) {
-				bucketState.pendingFilesPerCheckpoint.clear();
-			}
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Setters for User configuration values
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Sets the maximum bucket size in bytes.
-	 *
-	 * <p>
-	 * When a bucket part file becomes larger than this size a new bucket part file is started and
-	 * the old one is closed. The name of the bucket files depends on the {@link Bucketer}.
-	 *
-	 * @param batchSize The bucket part file size in bytes.
-	 */
-	public BucketingSink<T> setBatchSize(long batchSize) {
-		this.batchSize = batchSize;
-		return this;
-	}
-
-	/**
-	 * Sets the default time between checks for inactive buckets.
-	 *
-	 * @param interval The timeout, in milliseconds.
-	 */
-	public BucketingSink<T> setInactiveBucketCheckInterval(long interval) {
-		this.inactiveBucketCheckInterval = interval;
-		return this;
-	}
-
-	/**
-	 * Sets the default threshold for marking a bucket as inactive and closing its part files.
-	 * Buckets which haven't been written to for at least this period of time become inactive.
-	 *
-	 * @param threshold The timeout, in milliseconds.
-	 */
-	public BucketingSink<T> setInactiveBucketThreshold(long threshold) {
-		this.inactiveBucketThreshold = threshold;
-		return this;
-	}
-
-	/**
-	 * Sets the {@link Bucketer} to use for determining the bucket files to write to.
-	 *
-	 * @param bucketer The bucketer to use.
-	 */
-	public BucketingSink<T> setBucketer(Bucketer<T> bucketer) {
-		this.bucketer = bucketer;
-		return this;
-	}
-
-	/**
-	 * Sets the {@link Writer} to be used for writing the incoming elements to bucket files.
-	 *
-	 * @param writer The {@code Writer} to use.
-	 */
-	public BucketingSink<T> setWriter(Writer<T> writer) {
-		this.writerTemplate = writer;
-		return this;
-	}
-
-	/**
-	 * Sets the suffix of in-progress part files. The default is {@code "in-progress"}.
-	 */
-	public BucketingSink<T> setInProgressSuffix(String inProgressSuffix) {
-		this.inProgressSuffix = inProgressSuffix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of in-progress part files. The default is {@code "_"}.
-	 */
-	public BucketingSink<T> setInProgressPrefix(String inProgressPrefix) {
-		this.inProgressPrefix = inProgressPrefix;
-		return this;
-	}
-
-	/**
-	 * Sets the suffix of pending part files. The default is {@code ".pending"}.
-	 */
-	public BucketingSink<T> setPendingSuffix(String pendingSuffix) {
-		this.pendingSuffix = pendingSuffix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of pending part files. The default is {@code "_"}.
-	 */
-	public BucketingSink<T> setPendingPrefix(String pendingPrefix) {
-		this.pendingPrefix = pendingPrefix;
-		return this;
-	}
-
-	/**
-	 * Sets the suffix of valid-length files. The default is {@code ".valid-length"}.
-	 */
-	public BucketingSink<T> setValidLengthSuffix(String validLengthSuffix) {
-		this.validLengthSuffix = validLengthSuffix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of valid-length files. The default is {@code "_"}.
-	 */
-	public BucketingSink<T> setValidLengthPrefix(String validLengthPrefix) {
-		this.validLengthPrefix = validLengthPrefix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of part files.  The default is {@code "part"}.
-	 */
-	public BucketingSink<T> setPartPrefix(String partPrefix) {
-		this.partPrefix = partPrefix;
-		return this;
-	}
-
-	/**
-	 * Disable cleanup of leftover in-progress/pending files when the sink is opened.
-	 *
-	 * <p>
-	 * This should only be disabled if using the sink without checkpoints, to not remove
-	 * the files already in the directory.
-	 *
-	 * @deprecated This option is deprecated and remains only for backwards compatibility.
-	 * We do not clean up lingering files anymore.
-	 */
-	@Deprecated
-	public BucketingSink<T> disableCleanupOnOpen() {
-		return this;
-	}
-
-	/**
-	 * Sets the default timeout for asynchronous operations such as recoverLease and truncate.
-	 *
-	 * @param timeout The timeout, in milliseconds.
-	 */
-	public BucketingSink<T> setAsyncTimeout(long timeout) {
-		this.asyncTimeout = timeout;
-		return this;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Internal Classes
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * This is used during snapshot/restore to keep track of in-progress buckets.
-	 * For each bucket, we maintain a state.
-	 */
-	static final class State<T> implements Serializable {
-		private static final long serialVersionUID = 1L;
-
-		/**
-		 * For every bucket directory (key), we maintain a bucket state (value).
-		 */
-		final Map<String, BucketState<T>> bucketStates = new HashMap<>();
-
-		void addBucketState(Path bucketPath, BucketState<T> state) {
-			synchronized (bucketStates) {
-				bucketStates.put(bucketPath.toString(), state);
-			}
-		}
-
-		BucketState<T> getBucketState(Path bucketPath) {
-			synchronized (bucketStates) {
-				return bucketStates.get(bucketPath.toString());
-			}
-		}
-
-		@Override
-		public String toString() {
-			return bucketStates.toString();
-		}
-	}
-
-	/**
-	 * This is used for keeping track of the current in-progress buckets and files that we mark
-	 * for moving from pending to final location after we get a checkpoint-complete notification.
-	 */
-	static final class BucketState<T> implements Serializable {
-		private static final long serialVersionUID = 1L;
-
-		/**
-		 * The file that was in-progress when the last checkpoint occurred.
-		 */
-		String currentFile;
-
-		/**
-		 * The valid length of the in-progress file at the time of the last checkpoint.
-		 */
-		long currentFileValidLength = -1;
-
-		/**
-		 * The time this bucket was last written to.
-		 */
-		long lastWrittenToTime;
-
-		/**
-		 * Pending files that accumulated since the last checkpoint.
-		 */
-		List<String> pendingFiles = new ArrayList<>();
-
-		/**
-		 * When doing a checkpoint we move the pending files since the last checkpoint to this map
-		 * with the id of the checkpoint. When we get the checkpoint-complete notification we move
-		 * pending files of completed checkpoints to their final location.
-		 */
-		final Map<Long, List<String>> pendingFilesPerCheckpoint = new HashMap<>();
-
-		/**
-		 * For counting the part files inside a bucket directory. Part files follow the pattern
-		 * {@code "{part-prefix}-{subtask}-{count}"}. When creating new part files we increase the counter.
-		 */
-		private transient int partCounter;
-
-		/**
-		 * Tracks if the writer is currently opened or closed.
-		 */
-		private transient boolean isWriterOpen;
-
-		/**
-		 * The actual writer that we user for writing the part files.
-		 */
-		private transient Writer<T> writer;
-
-		@Override
-		public String toString() {
-			return
-				"In-progress=" + currentFile +
-					" validLength=" + currentFileValidLength +
-					" pendingForNextCheckpoint=" + pendingFiles +
-					" pendingForPrevCheckpoints=" + pendingFilesPerCheckpoint +
-					" lastModified@" + lastWrittenToTime;
-		}
-
-		BucketState(long lastWrittenToTime) {
-			this.lastWrittenToTime = lastWrittenToTime;
-		}
-	}
-}

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

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

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


[02/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
deleted file mode 100644
index a06fdca..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
+++ /dev/null
@@ -1,679 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.internals;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
-import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
-import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.InitialPosition;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
-import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
-import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
-import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
-import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
-import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
-import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
-import org.apache.flink.util.InstantiationUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import java.util.LinkedList;
-import java.util.List;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
- * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
- * <ul>
- *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
- *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
- *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
- *     		  to the same subset of shards even after restoring)</li>
- *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
- *     <li>3. subscribe to shards by creating a single thread for each shard</li>
- * </ul>
- *
- * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
- * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
- * by multiple threads, these operations should only be done using the handler methods provided in this class.
- */
-public class KinesisDataFetcher<T> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
-
-	// ------------------------------------------------------------------------
-	//  Consumer-wide settings
-	// ------------------------------------------------------------------------
-
-	/** Configuration properties for the Flink Kinesis Consumer */
-	private final Properties configProps;
-
-	/** The list of Kinesis streams that the consumer is subscribing to */
-	private final List<String> streams;
-
-	/**
-	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
-	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
-	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
-	 */
-	private final KinesisDeserializationSchema<T> deserializationSchema;
-
-	// ------------------------------------------------------------------------
-	//  Subtask-specific settings
-	// ------------------------------------------------------------------------
-
-	/** Runtime context of the subtask that this fetcher was created in */
-	private final RuntimeContext runtimeContext;
-
-	private final int totalNumberOfConsumerSubtasks;
-
-	private final int indexOfThisConsumerSubtask;
-
-	/**
-	 * This flag should be set by {@link FlinkKinesisConsumer} using
-	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
-	 */
-	private boolean isRestoredFromFailure;
-
-	// ------------------------------------------------------------------------
-	//  Executor services to run created threads
-	// ------------------------------------------------------------------------
-
-	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
-	private final ExecutorService shardConsumersExecutor;
-
-	// ------------------------------------------------------------------------
-	//  Managed state, accessed and updated across multiple threads
-	// ------------------------------------------------------------------------
-
-	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
-	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
-	 */
-	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
-
-	/**
-	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
-	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
-	 * the last processed sequence number of subscribed shards as they fetch and process records.
-	 *
-	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
-	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
-	 * all threads must use the following thread-safe methods this class provides to operate on this list:
-	 * <ul>
-	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
-	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
-	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, long, int, SequenceNumber)}</li>
-	 * </ul>
-	 */
-	private final List<KinesisStreamShardState> subscribedShardsState;
-
-	private final SourceFunction.SourceContext<T> sourceContext;
-
-	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
-	private final Object checkpointLock;
-
-	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
-	private final AtomicReference<Throwable> error;
-
-	/** The Kinesis proxy that the fetcher will be using to discover new shards */
-	private final KinesisProxyInterface kinesis;
-
-	/** Thread that executed runFetcher() */
-	private Thread mainThread;
-
-	/**
-	 * The current number of shards that are actively read by this fetcher.
-	 *
-	 * This value is updated in {@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)},
-	 * and {@link KinesisDataFetcher#updateState(int, SequenceNumber)}.
-	 */
-	private final AtomicInteger numberOfActiveShards = new AtomicInteger(0);
-
-	private volatile boolean running = true;
-
-	/**
-	 * Creates a Kinesis Data Fetcher.
-	 *
-	 * @param streams the streams to subscribe to
-	 * @param sourceContext context of the source function
-	 * @param runtimeContext this subtask's runtime context
-	 * @param configProps the consumer configuration properties
-	 * @param deserializationSchema deserialization schema
-	 */
-	public KinesisDataFetcher(List<String> streams,
-							SourceFunction.SourceContext<T> sourceContext,
-							RuntimeContext runtimeContext,
-							Properties configProps,
-							KinesisDeserializationSchema<T> deserializationSchema) {
-		this(streams,
-			sourceContext,
-			sourceContext.getCheckpointLock(),
-			runtimeContext,
-			configProps,
-			deserializationSchema,
-			new AtomicReference<Throwable>(),
-			new LinkedList<KinesisStreamShardState>(),
-			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
-			KinesisProxy.create(configProps));
-	}
-
-	/** This constructor is exposed for testing purposes */
-	protected KinesisDataFetcher(List<String> streams,
-								SourceFunction.SourceContext<T> sourceContext,
-								Object checkpointLock,
-								RuntimeContext runtimeContext,
-								Properties configProps,
-								KinesisDeserializationSchema<T> deserializationSchema,
-								AtomicReference<Throwable> error,
-								LinkedList<KinesisStreamShardState> subscribedShardsState,
-								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
-								KinesisProxyInterface kinesis) {
-		this.streams = checkNotNull(streams);
-		this.configProps = checkNotNull(configProps);
-		this.sourceContext = checkNotNull(sourceContext);
-		this.checkpointLock = checkNotNull(checkpointLock);
-		this.runtimeContext = checkNotNull(runtimeContext);
-		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
-		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
-		this.deserializationSchema = checkNotNull(deserializationSchema);
-		this.kinesis = checkNotNull(kinesis);
-
-		this.error = checkNotNull(error);
-		this.subscribedShardsState = checkNotNull(subscribedShardsState);
-		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
-
-		this.shardConsumersExecutor =
-			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
-	}
-
-	/**
-	 * Starts the fetcher. After starting the fetcher, it can only
-	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
-	 *
-	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
-	 */
-	public void runFetcher() throws Exception {
-
-		// check that we are running before proceeding
-		if (!running) {
-			return;
-		}
-
-		this.mainThread = Thread.currentThread();
-
-		// ------------------------------------------------------------------------
-		//  Procedures before starting the infinite while loop:
-		// ------------------------------------------------------------------------
-
-		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
-		//     and register them to the subscribedShardState list.
-		if (LOG.isDebugEnabled()) {
-			String logFormat = (!isRestoredFromFailure)
-				? "Subtask {} is trying to discover initial shards ..."
-				: "Subtask {} is trying to discover any new shards that were created while the consumer wasn't " +
-				"running due to failure ...";
-
-			LOG.debug(logFormat, indexOfThisConsumerSubtask);
-		}
-		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
-		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
-			// the starting state for new shards created while the consumer wasn't running depends on whether or not
-			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
-			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
-			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
-			InitialPosition initialPosition = InitialPosition.valueOf(configProps.getProperty(
-				ConsumerConfigConstants.STREAM_INITIAL_POSITION, ConsumerConfigConstants.DEFAULT_STREAM_INITIAL_POSITION));
-
-			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
-				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
-				: initialPosition.toSentinelSequenceNumber();
-
-			if (LOG.isInfoEnabled()) {
-				String logFormat = (!isRestoredFromFailure)
-					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
-					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't " +
-					"running due to failure, starting state set as sequence number {}";
-
-				LOG.info(logFormat, indexOfThisConsumerSubtask, shard.toString(), startingStateForNewShard.get());
-			}
-			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
-		}
-
-		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
-		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
-		boolean hasShards = false;
-		StringBuilder streamsWithNoShardsFound = new StringBuilder();
-		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
-			if (streamToLastDiscoveredShardEntry.getValue() != null) {
-				hasShards = true;
-			} else {
-				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
-			}
-		}
-
-		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
-			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
-				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
-		}
-
-		if (!hasShards) {
-			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
-		}
-
-		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
-		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
-		//     consumer using a restored state checkpoint
-		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
-			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
-
-			// only start a consuming thread if the seeded subscribed shard has not been completely read already
-			if (!seededShardState.getLastProcessedSequenceNum().equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
-
-				if (LOG.isInfoEnabled()) {
-					LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}",
-						indexOfThisConsumerSubtask, seededShardState.getKinesisStreamShard().toString(),
-						seededShardState.getLastProcessedSequenceNum(), seededStateIndex);
-					}
-
-				shardConsumersExecutor.submit(
-					new ShardConsumer<>(
-						this,
-						seededStateIndex,
-						subscribedShardsState.get(seededStateIndex).getKinesisStreamShard(),
-						subscribedShardsState.get(seededStateIndex).getLastProcessedSequenceNum()));
-			}
-		}
-
-		// ------------------------------------------------------------------------
-
-		// finally, start the infinite shard discovery and consumer launching loop;
-		// we will escape from this loop only when shutdownFetcher() or stopWithError() is called
-
-		final long discoveryIntervalMillis = Long.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS,
-				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
-
-		// FLINK-4341:
-		// For downstream operators that work on time (ex. window operators), we are required to emit a max value watermark
-		// for subtasks that won't continue to have shards to read from unless resharding happens in the future, otherwise
-		// the downstream watermarks would not advance, leading to unbounded accumulating state.
-		//
-		// The side-effect of this limitation is that on resharding, we must fail hard if the newly discovered shard
-		// is to be subscribed by a subtask that has previously emitted a max value watermark, otherwise the watermarks
-		// will be messed up.
-		//
-		// There are 2 cases were we need to either emit a max value watermark, or deliberately fail hard:
-		//  (a) if this subtask has no more shards to read from unless resharding happens in the future, we emit a max
-		//      value watermark. This case is encountered when 1) all previously read shards by this subtask were closed
-		//      due to resharding, 2) when this subtask was initially only subscribed to closed shards while the consumer
-		//      was told to start from TRIM_HORIZON, or 3) there was initially no shards for this subtask to read on startup.
-		//  (b) this subtask has discovered new shards to read from due to a reshard; if this subtask has already emitted
-		//      a max value watermark, we must deliberately fail hard to avoid messing up the watermarks. The new shards
-		//      will be subscribed by this subtask after restore as initial shards on startup.
-		//
-		// TODO: This is a temporary workaround until a min-watermark information service is available in the JobManager
-		// Please see FLINK-4341 for more detail
-
-		boolean emittedMaxValueWatermark = false;
-
-		if (this.numberOfActiveShards.get() == 0) {
-			// FLINK-4341 workaround case (a) - please see the above for details on this case
-			LOG.info("Subtask {} has no initial shards to read on startup; emitting max value watermark ...",
-				indexOfThisConsumerSubtask);
-			sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE));
-			emittedMaxValueWatermark = true;
-		}
-
-		while (running) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Subtask {} is trying to discover new shards that were created due to resharding ...",
-					indexOfThisConsumerSubtask);
-			}
-			List<KinesisStreamShard> newShardsDueToResharding = discoverNewShardsToSubscribe();
-
-			// -- NOTE: Potential race condition between newShardsDueToResharding and numberOfActiveShards --
-			// Since numberOfActiveShards is updated by parallel shard consuming threads in updateState(), there exists
-			// a race condition with the currently queried newShardsDueToResharding. Therefore, numberOfActiveShards
-			// may not correctly reflect the discover result in the below case determination. This may lead to incorrect
-			// case determination on the current discovery attempt, but can still be correctly handled on future attempts.
-			//
-			// Although this can be resolved by wrapping the current shard discovery attempt with the below
-			// case determination within a synchronized block on the checkpoint lock for atomicity, there will be
-			// considerable throughput performance regression as shard discovery is a remote call to AWS. Therefore,
-			// since the case determination is a temporary workaround for FLINK-4341, the race condition is tolerable as
-			// we can still eventually handle max value watermark emitting / deliberately failing on successive
-			// discovery attempts.
-
-			if (newShardsDueToResharding.size() == 0 && this.numberOfActiveShards.get() == 0 && !emittedMaxValueWatermark) {
-				// FLINK-4341 workaround case (a) - please see the above for details on this case
-				LOG.info("Subtask {} has completed reading all shards; emitting max value watermark ...",
-					indexOfThisConsumerSubtask);
-				sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE));
-				emittedMaxValueWatermark = true;
-			} else if (newShardsDueToResharding.size() > 0 && emittedMaxValueWatermark) {
-				// FLINK-4341 workaround case (b) - please see the above for details on this case
-				//
-				// Note that in the case where on resharding this subtask ceased to read all of it's previous shards
-				// but new shards is also to be subscribed by this subtask immediately after, emittedMaxValueWatermark
-				// will be false; this allows the fetcher to continue reading the new shards without failing on such cases.
-				// However, due to the race condition mentioned above, we might still fall into case (a) first, and
-				// then (b) on the next discovery attempt. Although the failure is ideally unnecessary, max value
-				// watermark emitting still remains to be correct.
-
-				LOG.warn("Subtask {} has discovered {} new shards to subscribe, but is failing hard to avoid messing" +
-						" up watermarks; the new shards will be subscribed by this subtask after restore ...",
-					indexOfThisConsumerSubtask, newShardsDueToResharding.size());
-				throw new RuntimeException("Deliberate failure to avoid messing up watermarks");
-			}
-
-			for (KinesisStreamShard shard : newShardsDueToResharding) {
-				// since there may be delay in discovering a new shard, all new shards due to
-				// resharding should be read starting from the earliest record possible
-				KinesisStreamShardState newShardState =
-					new KinesisStreamShardState(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get());
-				int newStateIndex = registerNewSubscribedShardState(newShardState);
-
-				if (LOG.isInfoEnabled()) {
-					LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " +
-							"the shard from sequence number {} with ShardConsumer {}",
-						indexOfThisConsumerSubtask, newShardState.getKinesisStreamShard().toString(),
-						newShardState.getLastProcessedSequenceNum(), newStateIndex);
-				}
-
-				shardConsumersExecutor.submit(
-					new ShardConsumer<>(
-						this,
-						newStateIndex,
-						newShardState.getKinesisStreamShard(),
-						newShardState.getLastProcessedSequenceNum()));
-			}
-
-			// we also check if we are running here so that we won't start the discovery sleep
-			// interval if the running flag was set to false during the middle of the while loop
-			if (running && discoveryIntervalMillis != 0) {
-				try {
-					Thread.sleep(discoveryIntervalMillis);
-				} catch (InterruptedException iex) {
-					// the sleep may be interrupted by shutdownFetcher()
-				}
-			}
-		}
-
-		// make sure all resources have been terminated before leaving
-		awaitTermination();
-
-		// any error thrown in the shard consumer threads will be thrown to the main thread
-		Throwable throwable = this.error.get();
-		if (throwable != null) {
-			if (throwable instanceof Exception) {
-				throw (Exception) throwable;
-			} else if (throwable instanceof Error) {
-				throw (Error) throwable;
-			} else {
-				throw new Exception(throwable);
-			}
-		}
-	}
-
-	/**
-	 * Creates a snapshot of the current last processed sequence numbers of each subscribed shard.
-	 *
-	 * @return state snapshot
-	 */
-	public HashMap<KinesisStreamShard, SequenceNumber> snapshotState() {
-		// this method assumes that the checkpoint lock is held
-		assert Thread.holdsLock(checkpointLock);
-
-		HashMap<KinesisStreamShard, SequenceNumber> stateSnapshot = new HashMap<>();
-		for (KinesisStreamShardState shardWithState : subscribedShardsState) {
-			stateSnapshot.put(shardWithState.getKinesisStreamShard(), shardWithState.getLastProcessedSequenceNum());
-		}
-		return stateSnapshot;
-	}
-
-	/**
-	 * Starts shutting down the fetcher. Must be called to allow {@link KinesisDataFetcher#runFetcher()} to complete.
-	 * Once called, the shutdown procedure will be executed and all shard consuming threads will be interrupted.
-	 */
-	public void shutdownFetcher() {
-		running = false;
-		mainThread.interrupt(); // the main thread may be sleeping for the discovery interval
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Shutting down the shard consumer threads of subtask {} ...", indexOfThisConsumerSubtask);
-		}
-		shardConsumersExecutor.shutdownNow();
-	}
-
-	/** After calling {@link KinesisDataFetcher#shutdownFetcher()}, this can be called to await the fetcher shutdown */
-	public void awaitTermination() throws InterruptedException {
-		while(!shardConsumersExecutor.isTerminated()) {
-			Thread.sleep(50);
-		}
-	}
-
-	/** Called by created threads to pass on errors. Only the first thrown error is set.
-	 * Once set, the shutdown process will be executed and all shard consuming threads will be interrupted. */
-	protected void stopWithError(Throwable throwable) {
-		if (this.error.compareAndSet(null, throwable)) {
-			shutdownFetcher();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Functions that update the subscribedStreamToLastDiscoveredShardIds state
-	// ------------------------------------------------------------------------
-
-	/** Updates the last discovered shard of a subscribed stream; only updates if the update is valid */
-	public void advanceLastDiscoveredShardOfStream(String stream, String shardId) {
-		String lastSeenShardIdOfStream = this.subscribedStreamsToLastDiscoveredShardIds.get(stream);
-
-		// the update is valid only if the given shard id is greater
-		// than the previous last seen shard id of the stream
-		if (lastSeenShardIdOfStream == null) {
-			// if not previously set, simply put as the last seen shard id
-			this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId);
-		} else if (KinesisStreamShard.compareShardIds(shardId, lastSeenShardIdOfStream) > 0) {
-			this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId);
-		}
-	}
-
-	/**
-	 * A utility function that does the following:
-	 *
-	 * 1. Find new shards for each stream that we haven't seen before
-	 * 2. For each new shard, determine whether this consumer subtask should subscribe to them;
-	 * 	  if yes, it is added to the returned list of shards
-	 * 3. Update the subscribedStreamsToLastDiscoveredShardIds state so that we won't get shards
-	 *    that we have already seen before the next time this function is called
-	 */
-	private List<KinesisStreamShard> discoverNewShardsToSubscribe() throws InterruptedException {
-
-		List<KinesisStreamShard> newShardsToSubscribe = new LinkedList<>();
-
-		GetShardListResult shardListResult = kinesis.getShardList(subscribedStreamsToLastDiscoveredShardIds);
-		if (shardListResult.hasRetrievedShards()) {
-			Set<String> streamsWithNewShards = shardListResult.getStreamsWithRetrievedShards();
-
-			for (String stream : streamsWithNewShards) {
-				List<KinesisStreamShard> newShardsOfStream = shardListResult.getRetrievedShardListOfStream(stream);
-				for (KinesisStreamShard newShard : newShardsOfStream) {
-					if (isThisSubtaskShouldSubscribeTo(newShard, totalNumberOfConsumerSubtasks, indexOfThisConsumerSubtask)) {
-						newShardsToSubscribe.add(newShard);
-					}
-				}
-
-				advanceLastDiscoveredShardOfStream(
-					stream, shardListResult.getLastSeenShardOfStream(stream).getShard().getShardId());
-			}
-		}
-
-		return newShardsToSubscribe;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Functions to get / set information about the consumer
-	// ------------------------------------------------------------------------
-
-	public void setIsRestoringFromFailure(boolean bool) {
-		this.isRestoredFromFailure = bool;
-	}
-
-	protected Properties getConsumerConfiguration() {
-		return configProps;
-	}
-
-	protected KinesisDeserializationSchema<T> getClonedDeserializationSchema() {
-		try {
-			return InstantiationUtil.clone(deserializationSchema, runtimeContext.getUserCodeClassLoader());
-		} catch (IOException | ClassNotFoundException ex) {
-			// this really shouldn't happen; simply wrap it around a runtime exception
-			throw new RuntimeException(ex);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Thread-safe operations for record emitting and shard state updating
-	//  that assure atomicity with respect to the checkpoint lock
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Atomic operation to collect a record and update state to the sequence number of the record.
-	 * This method is called by {@link ShardConsumer}s.
-	 *
-	 * @param record the record to collect
-	 * @param recordTimestamp timestamp to attach to the collected record
-	 * @param shardStateIndex index of the shard to update in subscribedShardsState;
-	 *                        this index should be the returned value from
-	 *                        {@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}, called
-	 *                        when the shard state was registered.
-	 * @param lastSequenceNumber the last sequence number value to update
-	 */
-	protected void emitRecordAndUpdateState(T record, long recordTimestamp, int shardStateIndex, SequenceNumber lastSequenceNumber) {
-		synchronized (checkpointLock) {
-			sourceContext.collectWithTimestamp(record, recordTimestamp);
-			updateState(shardStateIndex, lastSequenceNumber);
-		}
-	}
-
-	/**
-	 * Update the shard to last processed sequence number state.
-	 * This method is called by {@link ShardConsumer}s.
-	 *
-	 * @param shardStateIndex index of the shard to update in subscribedShardsState;
-	 *                        this index should be the returned value from
-	 *                        {@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}, called
-	 *                        when the shard state was registered.
-	 * @param lastSequenceNumber the last sequence number value to update
-	 */
-	protected void updateState(int shardStateIndex, SequenceNumber lastSequenceNumber) {
-		synchronized (checkpointLock) {
-			subscribedShardsState.get(shardStateIndex).setLastProcessedSequenceNum(lastSequenceNumber);
-
-			// if a shard's state is updated to be SENTINEL_SHARD_ENDING_SEQUENCE_NUM by its consumer thread,
-			// we've finished reading the shard and should determine it to be non-active
-			if (lastSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
-				this.numberOfActiveShards.decrementAndGet();
-				LOG.info("Subtask {} has reached the end of subscribed shard: {}",
-					indexOfThisConsumerSubtask, subscribedShardsState.get(shardStateIndex).getKinesisStreamShard());
-			}
-		}
-	}
-
-	/**
-	 * Register a new subscribed shard state.
-	 *
-	 * @param newSubscribedShardState the new shard state that this fetcher is to be subscribed to
-	 */
-	public int registerNewSubscribedShardState(KinesisStreamShardState newSubscribedShardState) {
-		synchronized (checkpointLock) {
-			subscribedShardsState.add(newSubscribedShardState);
-
-			// If a registered shard has initial state that is not SENTINEL_SHARD_ENDING_SEQUENCE_NUM (will be the case
-			// if the consumer had already finished reading a shard before we failed and restored), we determine that
-			// this subtask has a new active shard
-			if (!newSubscribedShardState.getLastProcessedSequenceNum().equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
-				this.numberOfActiveShards.incrementAndGet();
-			}
-
-			return subscribedShardsState.size()-1;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Miscellaneous utility functions
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Utility function to determine whether a shard should be subscribed by this consumer subtask.
-	 *
-	 * @param shard the shard to determine
-	 * @param totalNumberOfConsumerSubtasks total number of consumer subtasks
-	 * @param indexOfThisConsumerSubtask index of this consumer subtask
-	 */
-	private static boolean isThisSubtaskShouldSubscribeTo(KinesisStreamShard shard,
-														int totalNumberOfConsumerSubtasks,
-														int indexOfThisConsumerSubtask) {
-		return (Math.abs(shard.hashCode() % totalNumberOfConsumerSubtasks)) == indexOfThisConsumerSubtask;
-	}
-
-	private static ExecutorService createShardConsumersThreadPool(final String subtaskName) {
-		return Executors.newCachedThreadPool(new ThreadFactory() {
-			@Override
-			public Thread newThread(Runnable runnable) {
-				final AtomicLong threadCount = new AtomicLong(0);
-				Thread thread = new Thread(runnable);
-				thread.setName("shardConsumers-" + subtaskName + "-thread-" + threadCount.getAndIncrement());
-				thread.setDaemon(true);
-				return thread;
-			}
-		});
-	}
-
-	/**
-	 * Utility function to create an initial map of the last discovered shard id of each subscribed stream, set to null;
-	 * This is called in the constructor; correct values will be set later on by calling advanceLastDiscoveredShardOfStream()
-	 *
-	 * @param streams the list of subscribed streams
-	 * @return the initial map for subscribedStreamsToLastDiscoveredShardIds
-	 */
-	protected static HashMap<String, String> createInitialSubscribedStreamsToLastDiscoveredShardsState(List<String> streams) {
-		HashMap<String, String> initial = new HashMap<>();
-		for (String stream : streams) {
-			initial.put(stream, null);
-		}
-		return initial;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
deleted file mode 100644
index 612a4a7..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.internals;
-
-import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
-import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
-import com.amazonaws.services.kinesis.model.GetRecordsResult;
-import com.amazonaws.services.kinesis.model.Record;
-import com.amazonaws.services.kinesis.model.ShardIteratorType;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
-import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
-import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
-import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
-import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
-import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Thread that does the actual data pulling from AWS Kinesis shards. Each thread is in charge of one Kinesis shard only.
- */
-public class ShardConsumer<T> implements Runnable {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class);
-
-	private final KinesisDeserializationSchema<T> deserializer;
-
-	private final KinesisProxyInterface kinesis;
-
-	private final int subscribedShardStateIndex;
-
-	private final KinesisDataFetcher<T> fetcherRef;
-
-	private final KinesisStreamShard subscribedShard;
-
-	private final int maxNumberOfRecordsPerFetch;
-	private final long fetchIntervalMillis;
-
-	private SequenceNumber lastSequenceNum;
-
-	/**
-	 * Creates a shard consumer.
-	 *
-	 * @param fetcherRef reference to the owning fetcher
-	 * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
-	 * @param subscribedShard the shard this consumer is subscribed to
-	 * @param lastSequenceNum the sequence number in the shard to start consuming
-	 */
-	public ShardConsumer(KinesisDataFetcher<T> fetcherRef,
-						Integer subscribedShardStateIndex,
-						KinesisStreamShard subscribedShard,
-						SequenceNumber lastSequenceNum) {
-		this(fetcherRef,
-			subscribedShardStateIndex,
-			subscribedShard,
-			lastSequenceNum,
-			KinesisProxy.create(fetcherRef.getConsumerConfiguration()));
-	}
-
-	/** This constructor is exposed for testing purposes */
-	protected ShardConsumer(KinesisDataFetcher<T> fetcherRef,
-							Integer subscribedShardStateIndex,
-							KinesisStreamShard subscribedShard,
-							SequenceNumber lastSequenceNum,
-							KinesisProxyInterface kinesis) {
-		this.fetcherRef = checkNotNull(fetcherRef);
-		this.subscribedShardStateIndex = checkNotNull(subscribedShardStateIndex);
-		this.subscribedShard = checkNotNull(subscribedShard);
-		this.lastSequenceNum = checkNotNull(lastSequenceNum);
-		checkArgument(
-			!lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()),
-			"Should not start a ShardConsumer if the shard has already been completely read.");
-
-		this.deserializer = fetcherRef.getClonedDeserializationSchema();
-
-		Properties consumerConfig = fetcherRef.getConsumerConfiguration();
-		this.kinesis = kinesis;
-		this.maxNumberOfRecordsPerFetch = Integer.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_MAX,
-			Integer.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX)));
-		this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
-			Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)));
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public void run() {
-		String nextShardItr;
-
-		try {
-			// before infinitely looping, we set the initial nextShardItr appropriately
-
-			if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.get())) {
-				// if the shard is already closed, there will be no latest next record to get for this shard
-				if (subscribedShard.isClosed()) {
-					nextShardItr = null;
-				} else {
-					nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.LATEST.toString(), null);
-				}
-			} else if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get())) {
-				nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.TRIM_HORIZON.toString(), null);
-			} else if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
-				nextShardItr = null;
-			} else {
-				// we will be starting from an actual sequence number (due to restore from failure).
-				// if the last sequence number refers to an aggregated record, we need to clean up any dangling sub-records
-				// from the last aggregated record; otherwise, we can simply start iterating from the record right after.
-
-				if (lastSequenceNum.isAggregated()) {
-					String itrForLastAggregatedRecord =
-						kinesis.getShardIterator(subscribedShard, ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber());
-
-					// get only the last aggregated record
-					GetRecordsResult getRecordsResult = getRecords(itrForLastAggregatedRecord, 1);
-
-					List<UserRecord> fetchedRecords = deaggregateRecords(
-						getRecordsResult.getRecords(),
-						subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
-						subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
-
-					long lastSubSequenceNum = lastSequenceNum.getSubSequenceNumber();
-					for (UserRecord record : fetchedRecords) {
-						// we have found a dangling sub-record if it has a larger subsequence number
-						// than our last sequence number; if so, collect the record and update state
-						if (record.getSubSequenceNumber() > lastSubSequenceNum) {
-							deserializeRecordForCollectionAndUpdateState(record);
-						}
-					}
-
-					// set the nextShardItr so we can continue iterating in the next while loop
-					nextShardItr = getRecordsResult.getNextShardIterator();
-				} else {
-					// the last record was non-aggregated, so we can simply start from the next record
-					nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber());
-				}
-			}
-
-			while(isRunning()) {
-				if (nextShardItr == null) {
-					fetcherRef.updateState(subscribedShardStateIndex, SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get());
-
-					// we can close this consumer thread once we've reached the end of the subscribed shard
-					break;
-				} else {
-					if (fetchIntervalMillis != 0) {
-						Thread.sleep(fetchIntervalMillis);
-					}
-
-					GetRecordsResult getRecordsResult = getRecords(nextShardItr, maxNumberOfRecordsPerFetch);
-
-					// each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding
-					List<UserRecord> fetchedRecords = deaggregateRecords(
-						getRecordsResult.getRecords(),
-						subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
-						subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
-
-					for (UserRecord record : fetchedRecords) {
-						deserializeRecordForCollectionAndUpdateState(record);
-					}
-
-					nextShardItr = getRecordsResult.getNextShardIterator();
-				}
-			}
-		} catch (Throwable t) {
-			fetcherRef.stopWithError(t);
-		}
-	}
-
-	/**
-	 * The loop in run() checks this before fetching next batch of records. Since this runnable will be executed
-	 * by the ExecutorService {@link KinesisDataFetcher#shardConsumersExecutor}, the only way to close down this thread
-	 * would be by calling shutdownNow() on {@link KinesisDataFetcher#shardConsumersExecutor} and let the executor service
-	 * interrupt all currently running {@link ShardConsumer}s.
-	 */
-	private boolean isRunning() {
-		return !Thread.interrupted();
-	}
-
-	/**
-	 * Deserializes a record for collection, and accordingly updates the shard state in the fetcher. The last
-	 * successfully collected sequence number in this shard consumer is also updated so that
-	 * {@link ShardConsumer#getRecords(String, int)} may be able to use the correct sequence number to refresh shard
-	 * iterators if necessary.
-	 *
-	 * Note that the server-side Kinesis timestamp is attached to the record when collected. When the
-	 * user programs uses {@link TimeCharacteristic#EventTime}, this timestamp will be used by default.
-	 *
-	 * @param record record to deserialize and collect
-	 * @throws IOException
-	 */
-	private void deserializeRecordForCollectionAndUpdateState(UserRecord record)
-		throws IOException {
-		ByteBuffer recordData = record.getData();
-
-		byte[] dataBytes = new byte[recordData.remaining()];
-		recordData.get(dataBytes);
-
-		final long approxArrivalTimestamp = record.getApproximateArrivalTimestamp().getTime();
-
-		final T value = deserializer.deserialize(
-			dataBytes,
-			record.getPartitionKey(),
-			record.getSequenceNumber(),
-			approxArrivalTimestamp,
-			subscribedShard.getStreamName(),
-			subscribedShard.getShard().getShardId());
-
-		SequenceNumber collectedSequenceNumber = (record.isAggregated())
-			? new SequenceNumber(record.getSequenceNumber(), record.getSubSequenceNumber())
-			: new SequenceNumber(record.getSequenceNumber());
-
-		fetcherRef.emitRecordAndUpdateState(
-			value,
-			approxArrivalTimestamp,
-			subscribedShardStateIndex,
-			collectedSequenceNumber);
-
-		lastSequenceNum = collectedSequenceNumber;
-	}
-
-	/**
-	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
-	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
-	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
-	 * be used for the next call to this method.
-	 *
-	 * Note: it is important that this method is not called again before all the records from the last result have been
-	 * fully collected with {@link ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
-	 * {@link ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
-	 * incorrect shard iteration if the iterator had to be refreshed.
-	 *
-	 * @param shardItr shard iterator to use
-	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
-	 * @return get records result
-	 * @throws InterruptedException
-	 */
-	private GetRecordsResult getRecords(String shardItr, int maxNumberOfRecords) throws InterruptedException {
-		GetRecordsResult getRecordsResult = null;
-		while (getRecordsResult == null) {
-			try {
-				getRecordsResult = kinesis.getRecords(shardItr, maxNumberOfRecords);
-			} catch (ExpiredIteratorException eiEx) {
-				LOG.warn("Encountered an unexpected expired iterator {} for shard {};" +
-					" refreshing the iterator ...", shardItr, subscribedShard);
-				shardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber());
-
-				// sleep for the fetch interval before the next getRecords attempt with the refreshed iterator
-				if (fetchIntervalMillis != 0) {
-					Thread.sleep(fetchIntervalMillis);
-				}
-			}
-		}
-		return getRecordsResult;
-	}
-
-	@SuppressWarnings("unchecked")
-	protected static List<UserRecord> deaggregateRecords(List<Record> records, String startingHashKey, String endingHashKey) {
-		return UserRecord.deaggregate(records, new BigInteger(startingHashKey), new BigInteger(endingHashKey));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java
deleted file mode 100644
index 53ed11b..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.model;
-
-import com.amazonaws.services.kinesis.model.Shard;
-
-import java.io.Serializable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A serializable representation of a AWS Kinesis Stream shard. It is basically a wrapper class around the information
- * provided along with {@link com.amazonaws.services.kinesis.model.Shard}, with some extra utility methods to
- * determine whether or not a shard is closed and whether or not the shard is a result of parent shard splits or merges.
- */
-public class KinesisStreamShard implements Serializable {
-
-	private static final long serialVersionUID = -6004217801761077536L;
-
-	private final String streamName;
-	private final Shard shard;
-
-	private final int cachedHash;
-
-	/**
-	 * Create a new KinesisStreamShard
-	 *
-	 * @param streamName
-	 *           the name of the Kinesis stream that this shard belongs to
-	 * @param shard
-	 *           the actual AWS Shard instance that will be wrapped within this KinesisStreamShard
-	 */
-	public KinesisStreamShard(String streamName, Shard shard) {
-		this.streamName = checkNotNull(streamName);
-		this.shard = checkNotNull(shard);
-
-		// since our description of Kinesis Streams shards can be fully defined with the stream name and shard id,
-		// our hash doesn't need to use hash code of Amazon's description of Shards, which uses other info for calculation
-		int hash = 17;
-		hash = 37 * hash + streamName.hashCode();
-		hash = 37 * hash + shard.getShardId().hashCode();
-		this.cachedHash = hash;
-	}
-
-	public String getStreamName() {
-		return streamName;
-	}
-
-	public boolean isClosed() {
-		return (shard.getSequenceNumberRange().getEndingSequenceNumber() != null);
-	}
-
-	public Shard getShard() {
-		return shard;
-	}
-
-	@Override
-	public String toString() {
-		return "KinesisStreamShard{" +
-			"streamName='" + streamName + "'" +
-			", shard='" + shard.toString() + "'}";
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (!(obj instanceof KinesisStreamShard)) {
-			return false;
-		}
-
-		if (obj == this) {
-			return true;
-		}
-
-		KinesisStreamShard other = (KinesisStreamShard) obj;
-
-		return streamName.equals(other.getStreamName()) && shard.equals(other.getShard());
-	}
-
-	@Override
-	public int hashCode() {
-		return cachedHash;
-	}
-
-	/**
-	 * Utility function to compare two shard ids
-	 *
-	 * @param firstShardId first shard id to compare
-	 * @param secondShardId second shard id to compare
-	 * @return a value less than 0 if the first shard id is smaller than the second shard id,
-	 *         or a value larger than 0 the first shard is larger then the second shard id,
-	 *         or 0 if they are equal
-	 */
-	public static int compareShardIds(String firstShardId, String secondShardId) {
-		if (!isValidShardId(firstShardId)) {
-			throw new IllegalArgumentException("The first shard id has invalid format.");
-		}
-
-		if (!isValidShardId(secondShardId)) {
-			throw new IllegalArgumentException("The second shard id has invalid format.");
-		}
-
-		// digit segment of the shard id starts at index 8
-		return Long.compare(Long.parseLong(firstShardId.substring(8)), Long.parseLong(secondShardId.substring(8)));
-	}
-
-	/**
-	 * Checks if a shard id has valid format.
-	 * Kinesis stream shard ids have 12-digit numbers left-padded with 0's,
-	 * prefixed with "shardId-", ex. "shardId-000000000015".
-	 *
-	 * @param shardId the shard id to check
-	 * @return whether the shard id is valid
-	 */
-	public static boolean isValidShardId(String shardId) {
-		if (shardId == null) { return false; }
-		return shardId.matches("^shardId-\\d{12}");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java
deleted file mode 100644
index 00181da..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.model;
-
-/**
- * A wrapper class that bundles a {@link KinesisStreamShard} with its last processed sequence number.
- */
-public class KinesisStreamShardState {
-
-	private KinesisStreamShard kinesisStreamShard;
-	private SequenceNumber lastProcessedSequenceNum;
-
-	public KinesisStreamShardState(KinesisStreamShard kinesisStreamShard, SequenceNumber lastProcessedSequenceNum) {
-		this.kinesisStreamShard = kinesisStreamShard;
-		this.lastProcessedSequenceNum = lastProcessedSequenceNum;
-	}
-
-	public KinesisStreamShard getKinesisStreamShard() {
-		return this.kinesisStreamShard;
-	}
-
-	public SequenceNumber getLastProcessedSequenceNum() {
-		return this.lastProcessedSequenceNum;
-	}
-
-	public void setLastProcessedSequenceNum(SequenceNumber update) {
-		this.lastProcessedSequenceNum = update;
-	}
-
-	@Override
-	public String toString() {
-		return "KinesisStreamShardState{" +
-			"kinesisStreamShard='" + kinesisStreamShard.toString() + "'" +
-			", lastProcessedSequenceNumber='" + lastProcessedSequenceNum.toString() + "'}";
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (!(obj instanceof KinesisStreamShardState)) {
-			return false;
-		}
-
-		if (obj == this) {
-			return true;
-		}
-
-		KinesisStreamShardState other = (KinesisStreamShardState) obj;
-
-		return kinesisStreamShard.equals(other.getKinesisStreamShard()) && lastProcessedSequenceNum.equals(other.getLastProcessedSequenceNum());
-	}
-
-	@Override
-	public int hashCode() {
-		return 37 * (kinesisStreamShard.hashCode() + lastProcessedSequenceNum.hashCode());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java
deleted file mode 100644
index 8182201..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.model;
-
-import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
-import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
-
-/**
- * Special flag values for sequence numbers in shards to indicate special positions.
- * The value is initially set by {@link FlinkKinesisConsumer} when {@link KinesisDataFetcher}s are created.
- * The KinesisDataFetchers will use this value to determine how to retrieve the starting shard iterator from AWS Kinesis.
- */
-public enum SentinelSequenceNumber {
-
-	/** Flag value for shard's sequence numbers to indicate that the
-	 * shard should start to be read from the latest incoming records */
-	SENTINEL_LATEST_SEQUENCE_NUM( new SequenceNumber("LATEST_SEQUENCE_NUM") ),
-
-	/** Flag value for shard's sequence numbers to indicate that the shard should
-	 * start to be read from the earliest records that haven't expired yet */
-	SENTINEL_EARLIEST_SEQUENCE_NUM( new SequenceNumber("EARLIEST_SEQUENCE_NUM") ),
-
-	/** Flag value to indicate that we have already read the last record of this shard
-	 * (Note: Kinesis shards that have been closed due to a split or merge will have an ending data record) */
-	SENTINEL_SHARD_ENDING_SEQUENCE_NUM( new SequenceNumber("SHARD_ENDING_SEQUENCE_NUM") );
-
-	private SequenceNumber sentinel;
-
-	SentinelSequenceNumber(SequenceNumber sentinel) {
-		this.sentinel = sentinel;
-	}
-
-	public SequenceNumber get() {
-		return sentinel;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java
deleted file mode 100644
index 021f53f..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.model;
-
-import java.io.Serializable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A serializable representation of a Kinesis record's sequence number. It has two fields: the main sequence number,
- * and also a subsequence number. If this {@link SequenceNumber} is referring to an aggregated Kinesis record, the
- * subsequence number will be a non-negative value representing the order of the sub-record within the aggregation.
- */
-public class SequenceNumber implements Serializable {
-
-	private static final long serialVersionUID = 876972197938972667L;
-
-	private static final String DELIMITER = "-";
-
-	private final String sequenceNumber;
-	private final long subSequenceNumber;
-
-	private final int cachedHash;
-
-	/**
-	 * Create a new instance for a non-aggregated Kinesis record without a subsequence number.
-	 * @param sequenceNumber the sequence number
-	 */
-	public SequenceNumber(String sequenceNumber) {
-		this(sequenceNumber, -1);
-	}
-
-	/**
-	 * Create a new instance, with the specified sequence number and subsequence number.
-	 * To represent the sequence number for a non-aggregated Kinesis record, the subsequence number should be -1.
-	 * Otherwise, give a non-negative sequence number to represent an aggregated Kinesis record.
-	 *
-	 * @param sequenceNumber the sequence number
-	 * @param subSequenceNumber the subsequence number (-1 to represent non-aggregated Kinesis records)
-	 */
-	public SequenceNumber(String sequenceNumber, long subSequenceNumber) {
-		this.sequenceNumber = checkNotNull(sequenceNumber);
-		this.subSequenceNumber = subSequenceNumber;
-
-		this.cachedHash = 37 * (sequenceNumber.hashCode() + Long.valueOf(subSequenceNumber).hashCode());
-	}
-
-	public boolean isAggregated() {
-		return subSequenceNumber >= 0;
-	}
-
-	public String getSequenceNumber() {
-		return sequenceNumber;
-	}
-
-	public long getSubSequenceNumber() {
-		return subSequenceNumber;
-	}
-
-	@Override
-	public String toString() {
-		if (isAggregated()) {
-			return sequenceNumber + DELIMITER + subSequenceNumber;
-		} else {
-			return sequenceNumber;
-		}
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (!(obj instanceof SequenceNumber)) {
-			return false;
-		}
-
-		if (obj == this) {
-			return true;
-		}
-
-		SequenceNumber other = (SequenceNumber) obj;
-
-		return sequenceNumber.equals(other.getSequenceNumber())
-			&& (subSequenceNumber == other.getSubSequenceNumber());
-	}
-
-	@Override
-	public int hashCode() {
-		return cachedHash;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java
deleted file mode 100644
index 04b1654..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.proxy;
-
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
-
-import java.util.LinkedList;
-import java.util.List;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Basic model class to bundle the shards retrieved from Kinesis on a {@link KinesisProxyInterface#getShardList(Map)} call.
- */
-public class GetShardListResult {
-
-	private final Map<String, LinkedList<KinesisStreamShard>> streamsToRetrievedShardList = new HashMap<>();
-
-	public void addRetrievedShardToStream(String stream, KinesisStreamShard retrievedShard) {
-		if (!streamsToRetrievedShardList.containsKey(stream)) {
-			streamsToRetrievedShardList.put(stream, new LinkedList<KinesisStreamShard>());
-		}
-		streamsToRetrievedShardList.get(stream).add(retrievedShard);
-	}
-
-	public void addRetrievedShardsToStream(String stream, List<KinesisStreamShard> retrievedShards) {
-		if (retrievedShards.size() != 0) {
-			if (!streamsToRetrievedShardList.containsKey(stream)) {
-				streamsToRetrievedShardList.put(stream, new LinkedList<KinesisStreamShard>());
-			}
-			streamsToRetrievedShardList.get(stream).addAll(retrievedShards);
-		}
-	}
-
-	public List<KinesisStreamShard> getRetrievedShardListOfStream(String stream) {
-		if (!streamsToRetrievedShardList.containsKey(stream)) {
-			return null;
-		} else {
-			return streamsToRetrievedShardList.get(stream);
-		}
-	}
-
-	public KinesisStreamShard getLastSeenShardOfStream(String stream) {
-		if (!streamsToRetrievedShardList.containsKey(stream)) {
-			return null;
-		} else {
-			return streamsToRetrievedShardList.get(stream).getLast();
-		}
-	}
-
-	public boolean hasRetrievedShards() {
-		return !streamsToRetrievedShardList.isEmpty();
-	}
-
-	public Set<String> getStreamsWithRetrievedShards() {
-		return streamsToRetrievedShardList.keySet();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java
deleted file mode 100644
index 9ffc8e6..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java
+++ /dev/null
@@ -1,338 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.proxy;
-
-import com.amazonaws.services.kinesis.AmazonKinesisClient;
-import com.amazonaws.services.kinesis.model.DescribeStreamRequest;
-import com.amazonaws.services.kinesis.model.DescribeStreamResult;
-import com.amazonaws.services.kinesis.model.GetRecordsRequest;
-import com.amazonaws.services.kinesis.model.GetRecordsResult;
-import com.amazonaws.services.kinesis.model.GetShardIteratorResult;
-import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException;
-import com.amazonaws.services.kinesis.model.LimitExceededException;
-import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
-import com.amazonaws.services.kinesis.model.StreamStatus;
-import com.amazonaws.services.kinesis.model.Shard;
-import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
-import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.Map;
-import java.util.Random;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Kinesis proxy implementation - a utility class that is used as a proxy to make
- * calls to AWS Kinesis for several functions, such as getting a list of shards and
- * fetching a batch of data records starting from a specified record sequence number.
- *
- * NOTE:
- * In the AWS KCL library, there is a similar implementation - {@link com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxy}.
- * This implementation differs mainly in that we can make operations to arbitrary Kinesis streams, which is a needed
- * functionality for the Flink Kinesis Connecter since the consumer may simultaneously read from multiple Kinesis streams.
- */
-public class KinesisProxy implements KinesisProxyInterface {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KinesisProxy.class);
-
-	/** The actual Kinesis client from the AWS SDK that we will be using to make calls */
-	private final AmazonKinesisClient kinesisClient;
-
-	/** Random seed used to calculate backoff jitter for Kinesis operations */
-	private final static Random seed = new Random();
-
-	// ------------------------------------------------------------------------
-	//  describeStream() related performance settings
-	// ------------------------------------------------------------------------
-
-	/** Base backoff millis for the describe stream operation */
-	private final long describeStreamBaseBackoffMillis;
-
-	/** Maximum backoff millis for the describe stream operation */
-	private final long describeStreamMaxBackoffMillis;
-
-	/** Exponential backoff power constant for the describe stream operation */
-	private final double describeStreamExpConstant;
-
-	// ------------------------------------------------------------------------
-	//  getRecords() related performance settings
-	// ------------------------------------------------------------------------
-
-	/** Base backoff millis for the get records operation */
-	private final long getRecordsBaseBackoffMillis;
-
-	/** Maximum backoff millis for the get records operation */
-	private final long getRecordsMaxBackoffMillis;
-
-	/** Exponential backoff power constant for the get records operation */
-	private final double getRecordsExpConstant;
-
-	/** Maximum attempts for the get records operation */
-	private final int getRecordsMaxAttempts;
-
-	// ------------------------------------------------------------------------
-	//  getShardIterator() related performance settings
-	// ------------------------------------------------------------------------
-
-	/** Base backoff millis for the get shard iterator operation */
-	private final long getShardIteratorBaseBackoffMillis;
-
-	/** Maximum backoff millis for the get shard iterator operation */
-	private final long getShardIteratorMaxBackoffMillis;
-
-	/** Exponential backoff power constant for the get shard iterator operation */
-	private final double getShardIteratorExpConstant;
-
-	/** Maximum attempts for the get shard iterator operation */
-	private final int getShardIteratorMaxAttempts;
-
-	/**
-	 * Create a new KinesisProxy based on the supplied configuration properties
-	 *
-	 * @param configProps configuration properties containing AWS credential and AWS region info
-	 */
-	private KinesisProxy(Properties configProps) {
-		checkNotNull(configProps);
-
-		this.kinesisClient = AWSUtil.createKinesisClient(configProps);
-
-		this.describeStreamBaseBackoffMillis = Long.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE,
-				Long.toString(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE)));
-		this.describeStreamMaxBackoffMillis = Long.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX,
-				Long.toString(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX)));
-		this.describeStreamExpConstant = Double.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT,
-				Double.toString(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT)));
-
-		this.getRecordsBaseBackoffMillis = Long.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_BASE,
-				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE)));
-		this.getRecordsMaxBackoffMillis = Long.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_MAX,
-				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX)));
-		this.getRecordsExpConstant = Double.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT,
-				Double.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT)));
-		this.getRecordsMaxAttempts = Integer.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.SHARD_GETRECORDS_RETRIES,
-				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_RETRIES)));
-
-		this.getShardIteratorBaseBackoffMillis = Long.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_BASE,
-				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE)));
-		this.getShardIteratorMaxBackoffMillis = Long.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_MAX,
-				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX)));
-		this.getShardIteratorExpConstant = Double.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT,
-				Double.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT)));
-		this.getShardIteratorMaxAttempts = Integer.valueOf(
-			configProps.getProperty(
-				ConsumerConfigConstants.SHARD_GETITERATOR_RETRIES,
-				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETITERATOR_RETRIES)));
-
-	}
-
-	/**
-	 * Creates a Kinesis proxy.
-	 *
-	 * @param configProps configuration properties
-	 * @return the created kinesis proxy
-	 */
-	public static KinesisProxyInterface create(Properties configProps) {
-		return new KinesisProxy(configProps);
-	}
-
-	/**
-	 * {@inheritDoc}
-	 */
-	@Override
-	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException {
-		final GetRecordsRequest getRecordsRequest = new GetRecordsRequest();
-		getRecordsRequest.setShardIterator(shardIterator);
-		getRecordsRequest.setLimit(maxRecordsToGet);
-
-		GetRecordsResult getRecordsResult = null;
-
-		int attempt = 0;
-		while (attempt <= getRecordsMaxAttempts && getRecordsResult == null) {
-			try {
-				getRecordsResult = kinesisClient.getRecords(getRecordsRequest);
-			} catch (ProvisionedThroughputExceededException ex) {
-				long backoffMillis = fullJitterBackoff(
-					getRecordsBaseBackoffMillis, getRecordsMaxBackoffMillis, getRecordsExpConstant, attempt++);
-				LOG.warn("Got ProvisionedThroughputExceededException. Backing off for "
-					+ backoffMillis + " millis.");
-				Thread.sleep(backoffMillis);
-			}
-		}
-
-		if (getRecordsResult == null) {
-			throw new RuntimeException("Rate Exceeded for getRecords operation - all " + getRecordsMaxAttempts +
-				" retry attempts returned ProvisionedThroughputExceededException.");
-		}
-
-		return getRecordsResult;
-	}
-
-	/**
-	 * {@inheritDoc}
-	 */
-	@Override
-	public GetShardListResult getShardList(Map<String, String> streamNamesWithLastSeenShardIds) throws InterruptedException {
-		GetShardListResult result = new GetShardListResult();
-
-		for (Map.Entry<String,String> streamNameWithLastSeenShardId : streamNamesWithLastSeenShardIds.entrySet()) {
-			String stream = streamNameWithLastSeenShardId.getKey();
-			String lastSeenShardId = streamNameWithLastSeenShardId.getValue();
-			result.addRetrievedShardsToStream(stream, getShardsOfStream(stream, lastSeenShardId));
-		}
-		return result;
-	}
-
-	/**
-	 * {@inheritDoc}
-	 */
-	@Override
-	public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, @Nullable String startingSeqNum) throws InterruptedException {
-		GetShardIteratorResult getShardIteratorResult = null;
-
-		int attempt = 0;
-		while (attempt <= getShardIteratorMaxAttempts && getShardIteratorResult == null) {
-			try {
-				getShardIteratorResult =
-					kinesisClient.getShardIterator(shard.getStreamName(), shard.getShard().getShardId(), shardIteratorType, startingSeqNum);
-			} catch (ProvisionedThroughputExceededException ex) {
-				long backoffMillis = fullJitterBackoff(
-					getShardIteratorBaseBackoffMillis, getShardIteratorMaxBackoffMillis, getShardIteratorExpConstant, attempt++);
-				LOG.warn("Got ProvisionedThroughputExceededException. Backing off for "
-					+ backoffMillis + " millis.");
-				Thread.sleep(backoffMillis);
-			}
-		}
-
-		if (getShardIteratorResult == null) {
-			throw new RuntimeException("Rate Exceeded for getShardIterator operation - all " + getShardIteratorMaxAttempts +
-				" retry attempts returned ProvisionedThroughputExceededException.");
-		}
-		return getShardIteratorResult.getShardIterator();
-	}
-
-	private List<KinesisStreamShard> getShardsOfStream(String streamName, @Nullable String lastSeenShardId) throws InterruptedException {
-		List<KinesisStreamShard> shardsOfStream = new ArrayList<>();
-
-		DescribeStreamResult describeStreamResult;
-		do {
-			describeStreamResult = describeStream(streamName, lastSeenShardId);
-
-			List<Shard> shards = describeStreamResult.getStreamDescription().getShards();
-			for (Shard shard : shards) {
-				shardsOfStream.add(new KinesisStreamShard(streamName, shard));
-			}
-
-			if (shards.size() != 0) {
-				lastSeenShardId = shards.get(shards.size() - 1).getShardId();
-			}
-		} while (describeStreamResult.getStreamDescription().isHasMoreShards());
-
-		return shardsOfStream;
-	}
-
-	/**
-	 * Get metainfo for a Kinesis stream, which contains information about which shards this Kinesis stream possess.
-	 *
-	 * This method is using a "full jitter" approach described in AWS's article,
-	 * <a href="https://www.awsarchitectureblog.com/2015/03/backoff.html">"Exponential Backoff and Jitter"</a>.
-	 * This is necessary because concurrent calls will be made by all parallel subtask's fetcher. This
-	 * jitter backoff approach will help distribute calls across the fetchers over time.
-	 *
-	 * @param streamName the stream to describe
-	 * @param startShardId which shard to start with for this describe operation (earlier shard's infos will not appear in result)
-	 * @return the result of the describe stream operation
-	 */
-	private DescribeStreamResult describeStream(String streamName, @Nullable String startShardId) throws InterruptedException {
-		final DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest();
-		describeStreamRequest.setStreamName(streamName);
-		describeStreamRequest.setExclusiveStartShardId(startShardId);
-
-		DescribeStreamResult describeStreamResult = null;
-
-		// Call DescribeStream, with full-jitter backoff (if we get LimitExceededException).
-		int attemptCount = 0;
-		while (describeStreamResult == null) { // retry until we get a result
-			try {
-				describeStreamResult = kinesisClient.describeStream(describeStreamRequest);
-			} catch (LimitExceededException le) {
-				long backoffMillis = fullJitterBackoff(
-					describeStreamBaseBackoffMillis, describeStreamMaxBackoffMillis, describeStreamExpConstant, attemptCount++);
-				LOG.warn("Got LimitExceededException when describing stream " + streamName + ". Backing off for "
-					+ backoffMillis + " millis.");
-				Thread.sleep(backoffMillis);
-			} catch (ResourceNotFoundException re) {
-				throw new RuntimeException("Error while getting stream details", re);
-			}
-		}
-
-		String streamStatus = describeStreamResult.getStreamDescription().getStreamStatus();
-		if (!(streamStatus.equals(StreamStatus.ACTIVE.toString()) || streamStatus.equals(StreamStatus.UPDATING.toString()))) {
-			if (LOG.isWarnEnabled()) {
-				LOG.warn("The status of stream " + streamName + " is " + streamStatus + "; result of the current " +
-					"describeStream operation will not contain any shard information.");
-			}
-		}
-
-		// Kinesalite (mock implementation of Kinesis) does not correctly exclude shards before the exclusive
-		// start shard id in the returned shards list; check if we need to remove these erroneously returned shards
-		if (startShardId != null) {
-			List<Shard> shards = describeStreamResult.getStreamDescription().getShards();
-			Iterator<Shard> shardItr = shards.iterator();
-			while (shardItr.hasNext()) {
-				if (KinesisStreamShard.compareShardIds(shardItr.next().getShardId(), startShardId) <= 0) {
-					shardItr.remove();
-				}
-			}
-		}
-
-		return describeStreamResult;
-	}
-
-	private static long fullJitterBackoff(long base, long max, double power, int attempt) {
-		long exponentialBackoff = (long) Math.min(max, base * Math.pow(power, attempt));
-		return (long)(seed.nextDouble()*exponentialBackoff); // random jitter between 0 and the exponential backoff
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java
deleted file mode 100644
index 39ddc52..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.proxy;
-
-import com.amazonaws.services.kinesis.model.GetRecordsResult;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
-
-import java.util.Map;
-
-/**
- * Interface for a Kinesis proxy that operates on multiple Kinesis streams within the same AWS service region.
- */
-public interface KinesisProxyInterface {
-
-	/**
-	 * Get a shard iterator from the specified position in a shard.
-	 * The retrieved shard iterator can be used in {@link KinesisProxyInterface#getRecords(String, int)}}
-	 * to read data from the Kinesis shard.
-	 *
-	 * @param shard the shard to get the iterator
-	 * @param shardIteratorType the iterator type, defining how the shard is to be iterated
-	 *                          (one of: TRIM_HORIZON, LATEST, AT_SEQUENCE_NUMBER, AFTER_SEQUENCE_NUMBER)
-	 * @param startingSeqNum sequence number, must be null if shardIteratorType is TRIM_HORIZON or LATEST
-	 * @return shard iterator which can be used to read data from Kinesis
-	 * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the
-	 *                              operation has exceeded the rate limit; this exception will be thrown
-	 *                              if the backoff is interrupted.
-	 */
-	String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) throws InterruptedException;
-
-	/**
-	 * Get the next batch of data records using a specific shard iterator
-	 *
-	 * @param shardIterator a shard iterator that encodes info about which shard to read and where to start reading
-	 * @param maxRecordsToGet the maximum amount of records to retrieve for this batch
-	 * @return the batch of retrieved records, also with a shard iterator that can be used to get the next batch
-	 * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the
-	 *                              operation has exceeded the rate limit; this exception will be thrown
-	 *                              if the backoff is interrupted.
-	 */
-	GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException;
-
-	/**
-	 * Get shard list of multiple Kinesis streams, ignoring the
-	 * shards of each stream before a specified last seen shard id.
-	 *
-	 * @param streamNamesWithLastSeenShardIds a map with stream as key, and last seen shard id as value
-	 * @return result of the shard list query
-	 * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the
-	 *                              operation has exceeded the rate limit; this exception will be thrown
-	 *                              if the backoff is interrupted.
-	 */
-	GetShardListResult getShardList(Map<String,String> streamNamesWithLastSeenShardIds) throws InterruptedException;
-}


[47/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java b/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
deleted file mode 100644
index 35b0a7c..0000000
--- a/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
+++ /dev/null
@@ -1,289 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.addons.hbase;
-
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
-import org.apache.flink.api.common.io.RichInputFormat;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * {@link InputFormat} subclass that wraps the access for HTables.
- */
-public abstract class TableInputFormat<T extends Tuple> extends RichInputFormat<T, TableInputSplit> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(TableInputFormat.class);
-
-	/** helper variable to decide whether the input is exhausted or not */
-	private boolean endReached = false;
-
-	protected transient HTable table = null;
-	protected transient Scan scan = null;
-
-	/** HBase iterator wrapper */
-	private ResultScanner resultScanner = null;
-
-	private byte[] lastRow;
-	private int scannedRows;
-
-	/**
-	 * Returns an instance of Scan that retrieves the required subset of records from the HBase table.
-	 * @return The appropriate instance of Scan for this usecase.
-	 */
-	protected abstract Scan getScanner();
-
-	/**
-	 * What table is to be read.
-	 * Per instance of a TableInputFormat derivative only a single tablename is possible.
-	 * @return The name of the table
-	 */
-	protected abstract String getTableName();
-
-	/**
-	 * The output from HBase is always an instance of {@link Result}.
-	 * This method is to copy the data in the Result instance into the required {@link Tuple}
-	 * @param r The Result instance from HBase that needs to be converted
-	 * @return The approriate instance of {@link Tuple} that contains the needed information.
-	 */
-	protected abstract T mapResultToTuple(Result r);
-
-	/**
-	 * Creates a {@link Scan} object and opens the {@link HTable} connection.
-	 * These are opened here because they are needed in the createInputSplits
-	 * which is called before the openInputFormat method.
-	 * So the connection is opened in {@link #configure(Configuration)} and closed in {@link #closeInputFormat()}.
-	 *
-	 * @param parameters The configuration that is to be used
-	 * @see Configuration
-	 */
-	@Override
-	public void configure(Configuration parameters) {
-		table = createTable();
-		if (table != null) {
-			scan = getScanner();
-		}
-	}
-
-	/**
-	 * Create an {@link HTable} instance and set it into this format
-	 */
-	private HTable createTable() {
-		LOG.info("Initializing HBaseConfiguration");
-		//use files found in the classpath
-		org.apache.hadoop.conf.Configuration hConf = HBaseConfiguration.create();
-
-		try {
-			return new HTable(hConf, getTableName());
-		} catch (Exception e) {
-			LOG.error("Error instantiating a new HTable instance", e);
-		}
-		return null;
-	}
-
-	@Override
-	public void open(TableInputSplit split) throws IOException {
-		if (table == null) {
-			throw new IOException("The HBase table has not been opened!");
-		}
-		if (scan == null) {
-			throw new IOException("getScanner returned null");
-		}
-		if (split == null) {
-			throw new IOException("Input split is null!");
-		}
-
-		logSplitInfo("opening", split);
-		scan.setStartRow(split.getStartRow());
-		lastRow = split.getEndRow();
-		scan.setStopRow(lastRow);
-
-		resultScanner = table.getScanner(scan);
-		endReached = false;
-		scannedRows = 0;
-	}
-
-	@Override
-	public boolean reachedEnd() throws IOException {
-		return endReached;
-	}
-
-	@Override
-	public T nextRecord(T reuse) throws IOException {
-		if (resultScanner == null) {
-			throw new IOException("No table result scanner provided!");
-		}
-		try {
-			Result res = resultScanner.next();
-			if (res != null) {
-				scannedRows++;
-				lastRow = res.getRow();
-				return mapResultToTuple(res);
-			}
-		} catch (Exception e) {
-			resultScanner.close();
-			//workaround for timeout on scan
-			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
-			scan.setStartRow(lastRow);
-			resultScanner = table.getScanner(scan);
-			Result res = resultScanner.next();
-			if (res != null) {
-				scannedRows++;
-				lastRow = res.getRow();
-				return mapResultToTuple(res);
-			}
-		}
-
-		endReached = true;
-		return null;
-	}
-
-	@Override
-	public void close() throws IOException {
-		LOG.info("Closing split (scanned {} rows)", scannedRows);
-		lastRow = null;
-		try {
-			if (resultScanner != null) {
-				resultScanner.close();
-			}
-		} finally {
-			resultScanner = null;
-		}
-	}
-
-	@Override
-	public void closeInputFormat() throws IOException {
-		try {
-			if (table != null) {
-				table.close();
-			}
-		} finally {
-			table = null;
-		}
-	}
-
-	@Override
-	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
-		if (table == null) {
-			throw new IOException("The HBase table has not been opened!");
-		}
-		if (scan == null) {
-			throw new IOException("getScanner returned null");
-		}
-
-		//Gets the starting and ending row keys for every region in the currently open table
-		final Pair<byte[][], byte[][]> keys = table.getStartEndKeys();
-		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
-			throw new IOException("Expecting at least one region.");
-		}
-		final byte[] startRow = scan.getStartRow();
-		final byte[] stopRow = scan.getStopRow();
-		final boolean scanWithNoLowerBound = startRow.length == 0;
-		final boolean scanWithNoUpperBound = stopRow.length == 0;
-
-		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
-		for (int i = 0; i < keys.getFirst().length; i++) {
-			final byte[] startKey = keys.getFirst()[i];
-			final byte[] endKey = keys.getSecond()[i];
-			final String regionLocation = table.getRegionLocation(startKey, false).getHostnamePort();
-			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
-			if (!includeRegionInSplit(startKey, endKey)) {
-				continue;
-			}
-			//Finds the region on which the given row is being served
-			final String[] hosts = new String[]{regionLocation};
-
-			// determine if regions contains keys used by the scan
-			boolean isLastRegion = endKey.length == 0;
-			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
-				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
-
-				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
-				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
-					&& !isLastRegion ? endKey : stopRow;
-				int id = splits.size();
-				final TableInputSplit split = new TableInputSplit(id, hosts, table.getTableName(), splitStart, splitStop);
-				splits.add(split);
-			}
-		}
-		LOG.info("Created " + splits.size() + " splits");
-		for (TableInputSplit split : splits) {
-			logSplitInfo("created", split);
-		}
-		return splits.toArray(new TableInputSplit[0]);
-	}
-
-	private void logSplitInfo(String action, TableInputSplit split) {
-		int splitId = split.getSplitNumber();
-		String splitStart = Bytes.toString(split.getStartRow());
-		String splitEnd = Bytes.toString(split.getEndRow());
-		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
-		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
-		String[] hostnames = split.getHostnames();
-		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
-	}
-
-	/**
-	 * Test if the given region is to be included in the InputSplit while splitting the regions of a table.
-	 * <p>
-	 * This optimization is effective when there is a specific reasoning to exclude an entire region from the M-R job,
-	 * (and hence, not contributing to the InputSplit), given the start and end keys of the same. <br>
-	 * Useful when we need to remember the last-processed top record and revisit the [last, current) interval for M-R
-	 * processing, continuously. In addition to reducing InputSplits, reduces the load on the region server as well, due
-	 * to the ordering of the keys. <br>
-	 * <br>
-	 * Note: It is possible that <code>endKey.length() == 0 </code> , for the last (recent) region. <br>
-	 * Override this method, if you want to bulk exclude regions altogether from M-R. By default, no region is excluded(
-	 * i.e. all regions are included).
-	 *
-	 * @param startKey Start key of the region
-	 * @param endKey   End key of the region
-	 * @return true, if this region needs to be included as part of the input (default).
-	 */
-	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
-		return true;
-	}
-
-	@Override
-	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
-		return new LocatableInputSplitAssigner(inputSplits);
-	}
-
-	@Override
-	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java b/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
deleted file mode 100644
index 75f0b9b..0000000
--- a/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.addons.hbase;
-
-import org.apache.flink.core.io.LocatableInputSplit;
-
-/**
- * This class implements a input splits for HBase. Each table input split corresponds to a key range (low, high). All
- * references to row below refer to the key of the row.
- */
-public class TableInputSplit extends LocatableInputSplit {
-
-	private static final long serialVersionUID = 1L;
-
-	/** The name of the table to retrieve data from */
-	private final byte[] tableName;
-
-	/** The start row of the split. */
-	private final byte[] startRow;
-
-	/** The end row of the split. */
-	private final byte[] endRow;
-
-	/**
-	 * Creates a new table input split
-	 * 
-	 * @param splitNumber
-	 *        the number of the input split
-	 * @param hostnames
-	 *        the names of the hosts storing the data the input split refers to
-	 * @param tableName
-	 *        the name of the table to retrieve data from
-	 * @param startRow
-	 *        the start row of the split
-	 * @param endRow
-	 *        the end row of the split
-	 */
-	TableInputSplit(final int splitNumber, final String[] hostnames, final byte[] tableName, final byte[] startRow,
-			final byte[] endRow) {
-		super(splitNumber, hostnames);
-
-		this.tableName = tableName;
-		this.startRow = startRow;
-		this.endRow = endRow;
-	}
-
-	/**
-	 * Returns the table name.
-	 * 
-	 * @return The table name.
-	 */
-	public byte[] getTableName() {
-		return this.tableName;
-	}
-
-	/**
-	 * Returns the start row.
-	 * 
-	 * @return The start row.
-	 */
-	public byte[] getStartRow() {
-		return this.startRow;
-	}
-
-	/**
-	 * Returns the end row.
-	 * 
-	 * @return The end row.
-	 */
-	public byte[] getEndRow() {
-		return this.endRow;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseTestingClusterAutostarter.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseTestingClusterAutostarter.java b/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseTestingClusterAutostarter.java
deleted file mode 100644
index 3d9f672..0000000
--- a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseTestingClusterAutostarter.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/*
- * Copyright The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.addons.hbase;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MasterNotRunningException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.ScannerCallable;
-import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
-import org.apache.hadoop.hbase.ipc.RpcServer;
-import org.apache.log4j.Level;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * By using this class as the super class of a set of tests you will have a HBase testing
- * cluster available that is very suitable for writing tests for scanning and filtering against.
- * This is usable by any downstream application because the HBase cluster is 'injected' because
- * a dynamically generated hbase-site.xml is added to the classpath.
- * Because of this classpath manipulation it is not possible to start a second testing cluster in the same JVM.
- * So if you have this you should either put all hbase related tests in a single class or force surefire to
- * setup a new JVM for each testclass.
- * See: http://maven.apache.org/surefire/maven-surefire-plugin/examples/fork-options-and-parallel-execution.html
- */
-//
-// NOTE: The code in this file is based on code from the
-// Apache HBase project, licensed under the Apache License v 2.0
-//
-// https://github.com/apache/hbase/blob/master/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java
-//
-public class HBaseTestingClusterAutostarter implements Serializable {
-
-	private static final Log LOG = LogFactory.getLog(HBaseTestingClusterAutostarter.class);
-
-	private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-	private static HBaseAdmin admin = null;
-	private static List<TableName> createdTables = new ArrayList<>();
-
-	private static boolean alreadyRegisteredTestCluster = false;
-
-	protected static void createTable(TableName tableName, byte[] columnFamilyName, byte[][] splitKeys) {
-		LOG.info("HBase minicluster: Creating table " + tableName.getNameAsString());
-
-		assertNotNull("HBaseAdmin is not initialized successfully.", admin);
-		HTableDescriptor desc = new HTableDescriptor(tableName);
-		HColumnDescriptor colDef = new HColumnDescriptor(columnFamilyName);
-		desc.addFamily(colDef);
-
-		try {
-			admin.createTable(desc, splitKeys);
-			createdTables.add(tableName);
-			assertTrue("Fail to create the table", admin.tableExists(tableName));
-		} catch (IOException e) {
-			assertNull("Exception found while creating table", e);
-		}
-	}
-
-	protected static HTable openTable(TableName tableName) throws IOException {
-		HTable table = (HTable) admin.getConnection().getTable(tableName);
-		assertTrue("Fail to create the table", admin.tableExists(tableName));
-		return table;
-	}
-
-	private static void deleteTables() {
-		if (admin != null) {
-			for (TableName tableName : createdTables) {
-				try {
-					if (admin.tableExists(tableName)) {
-						admin.disableTable(tableName);
-						admin.deleteTable(tableName);
-					}
-				} catch (IOException e) {
-					assertNull("Exception found deleting the table", e);
-				}
-			}
-		}
-	}
-
-	private static void initialize(Configuration conf) {
-		conf = HBaseConfiguration.create(conf);
-		conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
-		try {
-			admin = TEST_UTIL.getHBaseAdmin();
-		} catch (MasterNotRunningException e) {
-			assertNull("Master is not running", e);
-		} catch (ZooKeeperConnectionException e) {
-			assertNull("Cannot connect to ZooKeeper", e);
-		} catch (IOException e) {
-			assertNull("IOException", e);
-		}
-	}
-
-	@BeforeClass
-	public static void setUp() throws Exception {
-		LOG.info("HBase minicluster: Starting");
-		((Log4JLogger) RpcServer.LOG).getLogger().setLevel(Level.ALL);
-		((Log4JLogger) AbstractRpcClient.LOG).getLogger().setLevel(Level.ALL);
-		((Log4JLogger) ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
-
-		TEST_UTIL.startMiniCluster(1);
-
-		// https://issues.apache.org/jira/browse/HBASE-11711
-		TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", -1);
-
-		// Make sure the zookeeper quorum value contains the right port number (varies per run).
-		TEST_UTIL.getConfiguration().set("hbase.zookeeper.quorum", "localhost:" + TEST_UTIL.getZkCluster().getClientPort());
-
-		initialize(TEST_UTIL.getConfiguration());
-		LOG.info("HBase minicluster: Running");
-	}
-
-	private static File hbaseSiteXmlDirectory;
-	private static File hbaseSiteXmlFile;
-
-	/**
-	 * This dynamically generates a hbase-site.xml file that is added to the classpath.
-	 * This way this HBaseMinicluster can be used by an unmodified application.
-	 * The downside is that this cannot be 'unloaded' so you can have only one per JVM.
-	 */
-	public static void registerHBaseMiniClusterInClasspath() {
-		if (alreadyRegisteredTestCluster) {
-			fail("You CANNOT register a second HBase Testing cluster in the classpath of the SAME JVM");
-		}
-		File baseDir = new File(System.getProperty("java.io.tmpdir", "/tmp/"));
-		hbaseSiteXmlDirectory = new File(baseDir, "unittest-hbase-minicluster-" + Math.abs(new Random().nextLong()) + "/");
-
-		if (!hbaseSiteXmlDirectory.mkdirs()) {
-			fail("Unable to create output directory " + hbaseSiteXmlDirectory + " for the HBase minicluster");
-		}
-
-		assertNotNull("The ZooKeeper for the HBase minicluster is missing", TEST_UTIL.getZkCluster());
-
-		createHBaseSiteXml(hbaseSiteXmlDirectory, TEST_UTIL.getConfiguration().get("hbase.zookeeper.quorum"));
-		addDirectoryToClassPath(hbaseSiteXmlDirectory);
-
-		// Avoid starting it again.
-		alreadyRegisteredTestCluster = true;
-	}
-
-	private static void createHBaseSiteXml(File hbaseSiteXmlDirectory, String zookeeperQuorum) {
-		hbaseSiteXmlFile = new File(hbaseSiteXmlDirectory, "hbase-site.xml");
-		// Create the hbase-site.xml file for this run.
-		try {
-			String hbaseSiteXml = "<?xml version=\"1.0\"?>\n" +
-				"<?xml-stylesheet type=\"text/xsl\" href=\"configuration.xsl\"?>\n" +
-				"<configuration>\n" +
-				"  <property>\n" +
-				"    <name>hbase.zookeeper.quorum</name>\n" +
-				"    <value>" + zookeeperQuorum + "</value>\n" +
-				"  </property>\n" +
-				"</configuration>";
-			OutputStream fos = new FileOutputStream(hbaseSiteXmlFile);
-			fos.write(hbaseSiteXml.getBytes(StandardCharsets.UTF_8));
-			fos.close();
-		} catch (IOException e) {
-			fail("Unable to create " + hbaseSiteXmlFile);
-		}
-	}
-
-	private static void addDirectoryToClassPath(File directory) {
-		try {
-			// Get the classloader actually used by HBaseConfiguration
-			ClassLoader classLoader = HBaseConfiguration.create().getClassLoader();
-			if (!(classLoader instanceof URLClassLoader)) {
-				fail("We should get a URLClassLoader");
-			}
-
-			// Make the addURL method accessible
-			Method method = URLClassLoader.class.getDeclaredMethod("addURL", URL.class);
-			method.setAccessible(true);
-
-			// Add the directory where we put the hbase-site.xml to the classpath
-			method.invoke(classLoader, directory.toURI().toURL());
-		} catch (MalformedURLException | NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
-			fail("Unable to add " + directory + " to classpath because of this exception: " + e.getMessage());
-		}
-	}
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		LOG.info("HBase minicluster: Shutting down");
-		deleteTables();
-		hbaseSiteXmlFile.delete();
-		hbaseSiteXmlDirectory.delete();
-		TEST_UTIL.shutdownMiniCluster();
-		LOG.info("HBase minicluster: Down");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/TableInputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/TableInputFormatITCase.java b/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/TableInputFormatITCase.java
deleted file mode 100644
index 3dddd88..0000000
--- a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/TableInputFormatITCase.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.addons.hbase;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TableInputFormatITCase extends HBaseTestingClusterAutostarter {
-	private static final String TEST_TABLE_NAME = "TableInputFormatTestTable";
-	private static final byte[] TEST_TABLE_FAMILY_NAME = "F".getBytes();
-	private static final byte[] TEST_TABLE_COLUMN_NAME = "Col".getBytes();
-
-	// These are the row ids AND also the values we will put in the test table
-	private static final String[] ROW_IDS = {"000", "111", "222", "333", "444", "555", "666", "777", "888", "999"};
-
-	@BeforeClass
-	public static void activateHBaseCluster(){
-		registerHBaseMiniClusterInClasspath();
-	}
-
-	@Before
-	public void createTestTable() throws IOException {
-		TableName tableName = TableName.valueOf(TEST_TABLE_NAME);
-		byte[][] splitKeys = {"0".getBytes(), "3".getBytes(), "6".getBytes(), "9".getBytes()};
-		createTable(tableName, TEST_TABLE_FAMILY_NAME, splitKeys);
-		HTable table = openTable(tableName);
-
-		for (String rowId : ROW_IDS) {
-			byte[] rowIdBytes = rowId.getBytes();
-			Put p = new Put(rowIdBytes);
-			// Use the rowId as the value to facilitate the testing better
-			p.add(TEST_TABLE_FAMILY_NAME, TEST_TABLE_COLUMN_NAME, rowIdBytes);
-			table.put(p);
-		}
-
-		table.close();
-	}
-
-	class InputFormatForTestTable extends TableInputFormat<Tuple1<String>> {
-		@Override
-		protected Scan getScanner() {
-			return new Scan();
-		}
-
-		@Override
-		protected String getTableName() {
-			return TEST_TABLE_NAME;
-		}
-
-		@Override
-		protected Tuple1<String> mapResultToTuple(Result r) {
-			return new Tuple1<>(new String(r.getValue(TEST_TABLE_FAMILY_NAME, TEST_TABLE_COLUMN_NAME)));
-		}
-	}
-
-	@Test
-	public void testTableInputFormat() {
-		ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment();
-		environment.setParallelism(1);
-
-		DataSet<String> resultDataSet =
-			environment.createInput(new InputFormatForTestTable()).map(new MapFunction<Tuple1<String>, String>() {
-				@Override
-				public String map(Tuple1<String> value) throws Exception {
-					return value.f0;
-				}
-			});
-
-		List<String> resultSet = new ArrayList<>();
-		resultDataSet.output(new LocalCollectionOutputFormat<>(resultSet));
-
-		try {
-			environment.execute("HBase InputFormat Test");
-		} catch (Exception e) {
-			Assert.fail("HBase InputFormat test failed. " + e.getMessage());
-		}
-
-		for (String rowId : ROW_IDS) {
-			assertTrue("Missing rowId from table: " + rowId, resultSet.contains(rowId));
-		}
-
-		assertEquals("The number of records is wrong.", ROW_IDS.length, resultSet.size());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java b/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java
deleted file mode 100644
index 8579dee..0000000
--- a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.addons.hbase.example;
-
-public class HBaseFlinkTestConstants {
-	
-	public static final byte[] CF_SOME = "someCf".getBytes();
-	public static final byte[] Q_SOME = "someQual".getBytes();
-	public static final String TEST_TABLE_NAME = "test-table";
-	public static final String TMP_DIR = "/tmp/test";
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java b/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java
deleted file mode 100644
index dccf876..0000000
--- a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.addons.hbase.example;
-
-import org.apache.flink.addons.hbase.TableInputFormat;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Simple stub for HBase DataSet read
- * 
- * To run the test first create the test table with hbase shell.
- * 
- * Use the following commands:
- * <ul>
- *     <li>create 'test-table', 'someCf'</li>
- *     <li>put 'test-table', '1', 'someCf:someQual', 'someString'</li>
- *     <li>put 'test-table', '2', 'someCf:someQual', 'anotherString'</li>
- * </ul>
- * 
- * The test should return just the first entry.
- * 
- */
-public class HBaseReadExample {
-	public static void main(String[] args) throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		@SuppressWarnings("serial")
-		DataSet<Tuple2<String, String>> hbaseDs = env.createInput(new TableInputFormat<Tuple2<String, String>>() {
-			
-				@Override
-				public String getTableName() {
-					return HBaseFlinkTestConstants.TEST_TABLE_NAME;
-				}
-
-				@Override
-				protected Scan getScanner() {
-					Scan scan = new Scan();
-					scan.addColumn(HBaseFlinkTestConstants.CF_SOME, HBaseFlinkTestConstants.Q_SOME);
-					return scan;
-				}
-
-				private Tuple2<String, String> reuse = new Tuple2<String, String>();
-				
-				@Override
-				protected Tuple2<String, String> mapResultToTuple(Result r) {
-					String key = Bytes.toString(r.getRow());
-					String val = Bytes.toString(r.getValue(HBaseFlinkTestConstants.CF_SOME, HBaseFlinkTestConstants.Q_SOME));
-					reuse.setField(key, 0);
-					reuse.setField(val, 1);
-					return reuse;
-				}
-		})
-		.filter(new FilterFunction<Tuple2<String,String>>() {
-
-			@Override
-			public boolean filter(Tuple2<String, String> t) throws Exception {
-				String val = t.getField(1);
-				if(val.startsWith("someStr"))
-					return true;
-				return false;
-			}
-		});
-		
-		hbaseDs.print();
-		
-		// kick off execution.
-		env.execute();
-				
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteExample.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteExample.java b/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteExample.java
deleted file mode 100644
index 483bdff..0000000
--- a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteExample.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.addons.hbase.example;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.Collector;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.mapreduce.TableOutputFormat;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-
-/**
- * Simple stub for HBase DataSet write
- * 
- * To run the test first create the test table with hbase shell.
- * 
- * Use the following commands:
- * <ul>
- *     <li>create 'test-table', 'someCf'</li>
- * </ul>
- * 
- */
-@SuppressWarnings("serial")
-public class HBaseWriteExample {
-	
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String[] args) throws Exception {
-
-		if(!parseParameters(args)) {
-			return;
-		}
-		
-		// set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		// get input data
-		DataSet<String> text = getTextDataSet(env);
-		
-		DataSet<Tuple2<String, Integer>> counts = 
-				// split up the lines in pairs (2-tuples) containing: (word,1)
-				text.flatMap(new Tokenizer())
-				// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0)
-				.sum(1);
-
-		// emit result
-		Job job = Job.getInstance();
-		job.getConfiguration().set(TableOutputFormat.OUTPUT_TABLE, outputTableName);
-		// TODO is "mapred.output.dir" really useful?
-		job.getConfiguration().set("mapred.output.dir",HBaseFlinkTestConstants.TMP_DIR);
-		counts.map(new RichMapFunction <Tuple2<String,Integer>, Tuple2<Text,Mutation>>() {
-			private transient Tuple2<Text, Mutation> reuse;
-
-			@Override
-			public void open(Configuration parameters) throws Exception {
-				super.open(parameters);
-				reuse = new Tuple2<Text, Mutation>();
-			}
-
-			@Override
-			public Tuple2<Text, Mutation> map(Tuple2<String, Integer> t) throws Exception {
-				reuse.f0 = new Text(t.f0);
-				Put put = new Put(t.f0.getBytes());
-				put.add(HBaseFlinkTestConstants.CF_SOME,HBaseFlinkTestConstants.Q_SOME, Bytes.toBytes(t.f1));
-				reuse.f1 = put;
-				return reuse;
-			}
-		}).output(new HadoopOutputFormat<Text, Mutation>(new TableOutputFormat<Text>(), job));
-		
-		// execute program
-		env.execute("WordCount (HBase sink) Example");
-	}
-	
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-	
-	/**
-	 * Implements the string tokenizer that splits sentences into words as a user-defined
-	 * FlatMapFunction. The function takes a line (String) and splits it into 
-	 * multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
-	 */
-	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
-
-		@Override
-		public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-			
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new Tuple2<String, Integer>(token, 1));
-				}
-			}
-		}
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputTableName = HBaseFlinkTestConstants.TEST_TABLE_NAME;
-	
-	private static boolean parseParameters(String[] args) {
-		
-		if(args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if(args.length == 2) {
-				textPath = args[0];
-				outputTableName = args[1];
-			} else {
-				System.err.println("Usage: HBaseWriteExample <text path> <output table>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing HBaseWriteExample example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from a file.");
-			System.out.println("  Usage: HBaseWriteExample <text path> <output table>");
-		}
-		return true;
-	}
-	
-	private static DataSet<String> getTextDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			// read the text file from given input path
-			return env.readTextFile(textPath);
-		} else {
-			// get default test text data
-			return getDefaultTextLineDataSet(env);
-		}
-	}
-	private static DataSet<String> getDefaultTextLineDataSet(ExecutionEnvironment env) {
-		return env.fromElements(WORDS);
-	}
-	private static final String[] WORDS = new String[] {
-		"To be, or not to be,--that is the question:--",
-		"Whether 'tis nobler in the mind to suffer",
-		"The slings and arrows of outrageous fortune",
-		"Or to take arms against a sea of troubles,",
-		"And by opposing end them?--To die,--to sleep,--",
-		"No more; and by a sleep to say we end",
-		"The heartache, and the thousand natural shocks",
-		"That flesh is heir to,--'tis a consummation",
-		"Devoutly to be wish'd. To die,--to sleep;--",
-		"To sleep! perchance to dream:--ay, there's the rub;",
-		"For in that sleep of death what dreams may come,",
-		"When we have shuffled off this mortal coil,",
-		"Must give us pause: there's the respect",
-		"That makes calamity of so long life;",
-		"For who would bear the whips and scorns of time,",
-		"The oppressor's wrong, the proud man's contumely,",
-		"The pangs of despis'd love, the law's delay,",
-		"The insolence of office, and the spurns",
-		"That patient merit of the unworthy takes,",
-		"When he himself might his quietus make",
-		"With a bare bodkin? who would these fardels bear,",
-		"To grunt and sweat under a weary life,",
-		"But that the dread of something after death,--",
-		"The undiscover'd country, from whose bourn",
-		"No traveller returns,--puzzles the will,",
-		"And makes us rather bear those ills we have",
-		"Than fly to others that we know not of?",
-		"Thus conscience does make cowards of us all;",
-		"And thus the native hue of resolution",
-		"Is sicklied o'er with the pale cast of thought;",
-		"And enterprises of great pith and moment,",
-		"With this regard, their currents turn awry,",
-		"And lose the name of action.--Soft you now!",
-		"The fair Ophelia!--Nymph, in thy orisons",
-		"Be all my sins remember'd."
-	};
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java b/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
deleted file mode 100644
index 05398db..0000000
--- a/flink-batch-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.addons.hbase.example;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * 
- * This is an example how to write streams into HBase. In this example the
- * stream will be written into a local Hbase but it is possible to adapt this
- * example for an HBase running in a cloud. You need a running local HBase with a
- * table "flinkExample" and a column "entry". If your HBase configuration does
- * not fit the hbase-site.xml in the resource folder then you gave to delete temporary this
- * hbase-site.xml to execute the example properly.
- * 
- */
-public class HBaseWriteStreamExample {
-
-	public static void main(String[] args) throws Exception {
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment
-				.getExecutionEnvironment();
-
-		// data stream with random numbers
-		DataStream<String> dataStream = env.addSource(new SourceFunction<String>() {
-			private static final long serialVersionUID = 1L;
-
-			private volatile boolean isRunning = true;
-
-			@Override
-			public void run(SourceContext<String> out) throws Exception {
-				while (isRunning) {
-					out.collect(String.valueOf(Math.floor(Math.random() * 100)));
-				}
-
-			}
-
-			@Override
-			public void cancel() {
-				isRunning = false;
-			}
-		});
-		dataStream.writeUsingOutputFormat(new HBaseOutputFormat());
-
-		env.execute();
-	}
-
-	/**
-	 * 
-	 * This class implements an OutputFormat for HBase
-	 *
-	 */
-	private static class HBaseOutputFormat implements OutputFormat<String> {
-
-		private org.apache.hadoop.conf.Configuration conf = null;
-		private HTable table = null;
-		private String taskNumber = null;
-		private int rowNumber = 0;
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void configure(Configuration parameters) {
-			conf = HBaseConfiguration.create();
-		}
-
-		@Override
-		public void open(int taskNumber, int numTasks) throws IOException {
-			table = new HTable(conf, "flinkExample");
-			this.taskNumber = String.valueOf(taskNumber);
-		}
-
-		@Override
-		public void writeRecord(String record) throws IOException {
-			Put put = new Put(Bytes.toBytes(taskNumber + rowNumber));
-			put.add(Bytes.toBytes("entry"), Bytes.toBytes("entry"),
-					Bytes.toBytes(rowNumber));
-			rowNumber++;
-			table.put(put);
-		}
-
-		@Override
-		public void close() throws IOException {
-			table.flushCommits();
-			table.close();
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hbase/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/test/resources/log4j-test.properties b/flink-batch-connectors/flink-hbase/src/test/resources/log4j-test.properties
deleted file mode 100644
index 804ff45..0000000
--- a/flink-batch-connectors/flink-hbase/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,23 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-# 
-# http://www.apache.org/licenses/LICENSE-2.0
-# 
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-
-log4j.rootLogger=DEBUG, stdout
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.Target=System.out
-log4j.appender.stdout.threshold=INFO
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ABSOLUTE} %-5p %30c{1}:%4L - %m%n

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hcatalog/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hcatalog/pom.xml b/flink-batch-connectors/flink-hcatalog/pom.xml
deleted file mode 100644
index 6889e5a..0000000
--- a/flink-batch-connectors/flink-hcatalog/pom.xml
+++ /dev/null
@@ -1,182 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-	
-	<modelVersion>4.0.0</modelVersion>
-	
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-batch-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-hcatalog</artifactId>
-	<name>flink-hcatalog</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-java</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hive.hcatalog</groupId>
-			<artifactId>hcatalog-core</artifactId>
-			<version>0.12.0</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.json</groupId>
-					<artifactId>json</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.scala-lang</groupId>
-			<artifactId>scala-library</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<!-- Scala Compiler -->
-			<plugin>
-				<groupId>net.alchim31.maven</groupId>
-				<artifactId>scala-maven-plugin</artifactId>
-				<executions>
-					<!-- Run scala compiler in the process-resources phase, so that dependencies on
-						scala classes can be resolved later in the (Java) compile phase -->
-					<execution>
-						<id>scala-compile-first</id>
-						<phase>process-resources</phase>
-						<goals>
-							<goal>compile</goal>
-						</goals>
-					</execution>
-
-					<!-- Run scala compiler in the process-test-resources phase, so that dependencies on
-						 scala classes can be resolved later in the (Java) test-compile phase -->
-					<execution>
-						<id>scala-test-compile</id>
-						<phase>process-test-resources</phase>
-						<goals>
-							<goal>testCompile</goal>
-						</goals>
-					</execution>
-				</executions>
-				<configuration>
-					<jvmArgs>
-						<jvmArg>-Xms128m</jvmArg>
-						<jvmArg>-Xmx512m</jvmArg>
-					</jvmArgs>
-				</configuration>
-			</plugin>
-
-			<!-- Eclipse Integration -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-eclipse-plugin</artifactId>
-				<version>2.8</version>
-				<configuration>
-					<downloadSources>true</downloadSources>
-					<projectnatures>
-						<projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
-						<projectnature>org.eclipse.jdt.core.javanature</projectnature>
-					</projectnatures>
-					<buildcommands>
-						<buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
-					</buildcommands>
-					<classpathContainers>
-						<classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
-						<classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
-					</classpathContainers>
-					<excludes>
-						<exclude>org.scala-lang:scala-library</exclude>
-						<exclude>org.scala-lang:scala-compiler</exclude>
-					</excludes>
-					<sourceIncludes>
-						<sourceInclude>**/*.scala</sourceInclude>
-						<sourceInclude>**/*.java</sourceInclude>
-					</sourceIncludes>
-				</configuration>
-			</plugin>
-
-			<!-- Adding scala source directories to build path -->
-			<plugin>
-				<groupId>org.codehaus.mojo</groupId>
-				<artifactId>build-helper-maven-plugin</artifactId>
-				<version>1.7</version>
-				<executions>
-					<!-- Add src/main/scala to eclipse build path -->
-					<execution>
-						<id>add-source</id>
-						<phase>generate-sources</phase>
-						<goals>
-							<goal>add-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>src/main/scala</source>
-							</sources>
-						</configuration>
-					</execution>
-					<!-- Add src/test/scala to eclipse build path -->
-					<execution>
-						<id>add-test-source</id>
-						<phase>generate-test-sources</phase>
-						<goals>
-							<goal>add-test-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>src/test/scala</source>
-							</sources>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-
-			<!-- Scala Code Style, most of the configuration done via plugin management -->
-			<plugin>
-				<groupId>org.scalastyle</groupId>
-				<artifactId>scalastyle-maven-plugin</artifactId>
-				<configuration>
-					<configLocation>${project.basedir}/../../tools/maven/scalastyle-config.xml</configLocation>
-				</configuration>
-			</plugin>
-
-		</plugins>
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java b/flink-batch-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java
deleted file mode 100644
index 859b706..0000000
--- a/flink-batch-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/HCatInputFormatBase.java
+++ /dev/null
@@ -1,410 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hcatalog;
-
-import org.apache.flink.api.common.io.RichInputFormat;
-import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.hadoop.mapreduce.utils.HadoopUtils;
-import org.apache.flink.api.java.hadoop.mapreduce.wrapper.HadoopInputSplit;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.WritableTypeInfo;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hive.hcatalog.common.HCatException;
-import org.apache.hive.hcatalog.common.HCatUtil;
-import org.apache.hive.hcatalog.data.DefaultHCatRecord;
-import org.apache.hive.hcatalog.data.HCatRecord;
-import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
-import org.apache.hive.hcatalog.data.schema.HCatSchema;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A InputFormat to read from HCatalog tables.
- * The InputFormat supports projection (selection and order of fields) and partition filters.
- *
- * Data can be returned as {@link org.apache.hive.hcatalog.data.HCatRecord} or Flink-native tuple.
- *
- * Note: Flink tuples might only support a limited number of fields (depending on the API).
- *
- * @param <T>
- */
-public abstract class HCatInputFormatBase<T> extends RichInputFormat<T, HadoopInputSplit> implements ResultTypeQueryable<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private Configuration configuration;
-
-	private org.apache.hive.hcatalog.mapreduce.HCatInputFormat hCatInputFormat;
-	private RecordReader<WritableComparable, HCatRecord> recordReader;
-	private boolean fetched = false;
-	private boolean hasNext;
-
-	protected String[] fieldNames = new String[0];
-	protected HCatSchema outputSchema;
-
-	private TypeInformation<T> resultType;
-
-	public HCatInputFormatBase() { }
-
-	/**
-	 * Creates a HCatInputFormat for the given database and table.
-	 * By default, the InputFormat returns {@link org.apache.hive.hcatalog.data.HCatRecord}.
-	 * The return type of the InputFormat can be changed to Flink-native tuples by calling
-	 * {@link HCatInputFormatBase#asFlinkTuples()}.
-	 *
-	 * @param database The name of the database to read from.
-	 * @param table The name of the table to read.
-	 * @throws java.io.IOException
-	 */
-	public HCatInputFormatBase(String database, String table) throws IOException {
-		this(database, table, new Configuration());
-	}
-
-	/**
-	 * Creates a HCatInputFormat for the given database, table, and
-	 * {@link org.apache.hadoop.conf.Configuration}.
-	 * By default, the InputFormat returns {@link org.apache.hive.hcatalog.data.HCatRecord}.
-	 * The return type of the InputFormat can be changed to Flink-native tuples by calling
-	 * {@link HCatInputFormatBase#asFlinkTuples()}.
-	 *
-	 * @param database The name of the database to read from.
-	 * @param table The name of the table to read.
-	 * @param config The Configuration for the InputFormat.
-	 * @throws java.io.IOException
-	 */
-	public HCatInputFormatBase(String database, String table, Configuration config) throws IOException {
-		super();
-		this.configuration = config;
-		HadoopUtils.mergeHadoopConf(this.configuration);
-
-		this.hCatInputFormat = org.apache.hive.hcatalog.mapreduce.HCatInputFormat.setInput(this.configuration, database, table);
-		this.outputSchema = org.apache.hive.hcatalog.mapreduce.HCatInputFormat.getTableSchema(this.configuration);
-
-		// configure output schema of HCatFormat
-		configuration.set("mapreduce.lib.hcat.output.schema", HCatUtil.serialize(outputSchema));
-		// set type information
-		this.resultType = new WritableTypeInfo(DefaultHCatRecord.class);
-	}
-
-	/**
-	 * Specifies the fields which are returned by the InputFormat and their order.
-	 *
-	 * @param fields The fields and their order which are returned by the InputFormat.
-	 * @return This InputFormat with specified return fields.
-	 * @throws java.io.IOException
-	 */
-	public HCatInputFormatBase<T> getFields(String... fields) throws IOException {
-
-		// build output schema
-		ArrayList<HCatFieldSchema> fieldSchemas = new ArrayList<HCatFieldSchema>(fields.length);
-		for(String field : fields) {
-			fieldSchemas.add(this.outputSchema.get(field));
-		}
-		this.outputSchema = new HCatSchema(fieldSchemas);
-
-		// update output schema configuration
-		configuration.set("mapreduce.lib.hcat.output.schema", HCatUtil.serialize(outputSchema));
-
-		return this;
-	}
-
-	/**
-	 * Specifies a SQL-like filter condition on the table's partition columns.
-	 * Filter conditions on non-partition columns are invalid.
-	 * A partition filter can significantly reduce the amount of data to be read.
-	 *
-	 * @param filter A SQL-like filter condition on the table's partition columns.
-	 * @return This InputFormat with specified partition filter.
-	 * @throws java.io.IOException
-	 */
-	public HCatInputFormatBase<T> withFilter(String filter) throws IOException {
-
-		// set filter
-		this.hCatInputFormat.setFilter(filter);
-
-		return this;
-	}
-
-	/**
-	 * Specifies that the InputFormat returns Flink tuples instead of
-	 * {@link org.apache.hive.hcatalog.data.HCatRecord}.
-	 *
-	 * Note: Flink tuples might only support a limited number of fields (depending on the API).
-	 *
-	 * @return This InputFormat.
-	 * @throws org.apache.hive.hcatalog.common.HCatException
-	 */
-	public HCatInputFormatBase<T> asFlinkTuples() throws HCatException {
-
-		// build type information
-		int numFields = outputSchema.getFields().size();
-		if(numFields > this.getMaxFlinkTupleSize()) {
-			throw new IllegalArgumentException("Only up to "+this.getMaxFlinkTupleSize()+
-					" fields can be returned as Flink tuples.");
-		}
-
-		TypeInformation[] fieldTypes = new TypeInformation[numFields];
-		fieldNames = new String[numFields];
-		for (String fieldName : outputSchema.getFieldNames()) {
-			HCatFieldSchema field = outputSchema.get(fieldName);
-
-			int fieldPos = outputSchema.getPosition(fieldName);
-			TypeInformation fieldType = getFieldType(field);
-
-			fieldTypes[fieldPos] = fieldType;
-			fieldNames[fieldPos] = fieldName;
-
-		}
-		this.resultType = new TupleTypeInfo(fieldTypes);
-
-		return this;
-	}
-
-	protected abstract int getMaxFlinkTupleSize();
-
-	private TypeInformation getFieldType(HCatFieldSchema fieldSchema) {
-
-		switch(fieldSchema.getType()) {
-			case INT:
-				return BasicTypeInfo.INT_TYPE_INFO;
-			case TINYINT:
-				return BasicTypeInfo.BYTE_TYPE_INFO;
-			case SMALLINT:
-				return BasicTypeInfo.SHORT_TYPE_INFO;
-			case BIGINT:
-				return BasicTypeInfo.LONG_TYPE_INFO;
-			case BOOLEAN:
-				return BasicTypeInfo.BOOLEAN_TYPE_INFO;
-			case FLOAT:
-				return BasicTypeInfo.FLOAT_TYPE_INFO;
-			case DOUBLE:
-				return BasicTypeInfo.DOUBLE_TYPE_INFO;
-			case STRING:
-				return BasicTypeInfo.STRING_TYPE_INFO;
-			case BINARY:
-				return PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO;
-			case ARRAY:
-				return new GenericTypeInfo(List.class);
-			case MAP:
-				return new GenericTypeInfo(Map.class);
-			case STRUCT:
-				return new GenericTypeInfo(List.class);
-			default:
-				throw new IllegalArgumentException("Unknown data type \""+fieldSchema.getType()+"\" encountered.");
-		}
-	}
-
-	/**
-	 * Returns the {@link org.apache.hadoop.conf.Configuration} of the HCatInputFormat.
-	 *
-	 * @return The Configuration of the HCatInputFormat.
-	 */
-	public Configuration getConfiguration() {
-		return this.configuration;
-	}
-
-	/**
-	 * Returns the {@link org.apache.hive.hcatalog.data.schema.HCatSchema} of the {@link org.apache.hive.hcatalog.data.HCatRecord}
-	 * returned by this InputFormat.
-	 *
-	 * @return The HCatSchema of the HCatRecords returned by this InputFormat.
-	 */
-	public HCatSchema getOutputSchema() {
-		return this.outputSchema;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  InputFormat
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public void configure(org.apache.flink.configuration.Configuration parameters) {
-		// nothing to do
-	}
-
-	@Override
-	public BaseStatistics getStatistics(BaseStatistics cachedStats) throws IOException {
-		// no statistics provided at the moment
-		return null;
-	}
-
-	@Override
-	public HadoopInputSplit[] createInputSplits(int minNumSplits)
-			throws IOException {
-		configuration.setInt("mapreduce.input.fileinputformat.split.minsize", minNumSplits);
-
-		JobContext jobContext = null;
-		try {
-			jobContext = HadoopUtils.instantiateJobContext(configuration, new JobID());
-		} catch (Exception e) {
-			throw new RuntimeException(e);
-		}
-
-		List<InputSplit> splits;
-		try {
-			splits = this.hCatInputFormat.getSplits(jobContext);
-		} catch (InterruptedException e) {
-			throw new IOException("Could not get Splits.", e);
-		}
-		HadoopInputSplit[] hadoopInputSplits = new HadoopInputSplit[splits.size()];
-
-		for(int i = 0; i < hadoopInputSplits.length; i++){
-			hadoopInputSplits[i] = new HadoopInputSplit(i, splits.get(i), jobContext);
-		}
-		return hadoopInputSplits;
-	}
-
-	@Override
-	public InputSplitAssigner getInputSplitAssigner(HadoopInputSplit[] inputSplits) {
-		return new LocatableInputSplitAssigner(inputSplits);
-	}
-
-	@Override
-	public void open(HadoopInputSplit split) throws IOException {
-		TaskAttemptContext context = null;
-		try {
-			context = HadoopUtils.instantiateTaskAttemptContext(configuration, new TaskAttemptID());
-		} catch(Exception e) {
-			throw new RuntimeException(e);
-		}
-
-		try {
-			this.recordReader = this.hCatInputFormat
-					.createRecordReader(split.getHadoopInputSplit(), context);
-			this.recordReader.initialize(split.getHadoopInputSplit(), context);
-		} catch (InterruptedException e) {
-			throw new IOException("Could not create RecordReader.", e);
-		} finally {
-			this.fetched = false;
-		}
-	}
-
-	@Override
-	public boolean reachedEnd() throws IOException {
-		if(!this.fetched) {
-			fetchNext();
-		}
-		return !this.hasNext;
-	}
-
-	private void fetchNext() throws IOException {
-		try {
-			this.hasNext = this.recordReader.nextKeyValue();
-		} catch (InterruptedException e) {
-			throw new IOException("Could not fetch next KeyValue pair.", e);
-		} finally {
-			this.fetched = true;
-		}
-	}
-
-	@Override
-	public T nextRecord(T record) throws IOException {
-		if(!this.fetched) {
-			// first record
-			fetchNext();
-		}
-		if(!this.hasNext) {
-			return null;
-		}
-		try {
-
-			// get next HCatRecord
-			HCatRecord v = this.recordReader.getCurrentValue();
-			this.fetched = false;
-
-			if(this.fieldNames.length > 0) {
-				// return as Flink tuple
-				return this.buildFlinkTuple(record, v);
-
-			} else {
-				// return as HCatRecord
-				return (T)v;
-			}
-
-		} catch (InterruptedException e) {
-			throw new IOException("Could not get next record.", e);
-		}
-	}
-
-	protected abstract T buildFlinkTuple(T t, HCatRecord record) throws HCatException;
-
-	@Override
-	public void close() throws IOException {
-		this.recordReader.close();
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Custom de/serialization methods
-	// --------------------------------------------------------------------------------------------
-
-	private void writeObject(ObjectOutputStream out) throws IOException {
-		out.writeInt(this.fieldNames.length);
-		for(String fieldName : this.fieldNames) {
-			out.writeUTF(fieldName);
-		}
-		this.configuration.write(out);
-	}
-
-	@SuppressWarnings("unchecked")
-	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-		this.fieldNames = new String[in.readInt()];
-		for(int i=0; i<this.fieldNames.length; i++) {
-			this.fieldNames[i] = in.readUTF();
-		}
-
-		Configuration configuration = new Configuration();
-		configuration.readFields(in);
-
-		if(this.configuration == null) {
-			this.configuration = configuration;
-		}
-
-		this.hCatInputFormat = new org.apache.hive.hcatalog.mapreduce.HCatInputFormat();
-		this.outputSchema = (HCatSchema)HCatUtil.deserialize(this.configuration.get("mapreduce.lib.hcat.output.schema"));
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Result type business
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public TypeInformation<T> getProducedType() {
-		return this.resultType;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/java/HCatInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/java/HCatInputFormat.java b/flink-batch-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/java/HCatInputFormat.java
deleted file mode 100644
index 46f3cd5..0000000
--- a/flink-batch-connectors/flink-hcatalog/src/main/java/org/apache/flink/hcatalog/java/HCatInputFormat.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hcatalog.java;
-
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.hcatalog.HCatInputFormatBase;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hive.hcatalog.common.HCatException;
-import org.apache.hive.hcatalog.data.HCatRecord;
-
-/**
- * A InputFormat to read from HCatalog tables.
- * The InputFormat supports projection (selection and order of fields) and partition filters.
- *
- * Data can be returned as {@link HCatRecord} or Flink {@link org.apache.flink.api.java.tuple.Tuple}.
- * Flink tuples support only up to 25 fields.
- *
- * @param <T>
- */
-public class HCatInputFormat<T> extends HCatInputFormatBase<T> {
-	private static final long serialVersionUID = 1L;
-
-	public HCatInputFormat() {}
-
-	public HCatInputFormat(String database, String table) throws Exception {
-		super(database, table);
-	}
-
-	public HCatInputFormat(String database, String table, Configuration config) throws Exception {
-		super(database, table, config);
-	}
-
-
-	@Override
-	protected int getMaxFlinkTupleSize() {
-		return 25;
-	}
-
-	@Override
-	protected T buildFlinkTuple(T t, HCatRecord record) throws HCatException {
-
-		Tuple tuple = (Tuple)t;
-
-		// Extract all fields from HCatRecord
-		for(int i=0; i < this.fieldNames.length; i++) {
-
-			// get field value
-			Object o = record.get(this.fieldNames[i], this.outputSchema);
-
-			// Set field value in Flink tuple.
-			// Partition columns are returned as String and
-			//   need to be converted to original type.
-			switch(this.outputSchema.get(i).getType()) {
-				case INT:
-					if(o instanceof String) {
-						tuple.setField(Integer.parseInt((String) o), i);
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case TINYINT:
-					if(o instanceof String) {
-						tuple.setField(Byte.parseByte((String) o), i);
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case SMALLINT:
-					if(o instanceof String) {
-						tuple.setField(Short.parseShort((String) o), i);
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case BIGINT:
-					if(o instanceof String) {
-						tuple.setField(Long.parseLong((String) o), i);
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case BOOLEAN:
-					if(o instanceof String) {
-						tuple.setField(Boolean.parseBoolean((String) o), i);
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case FLOAT:
-					if(o instanceof String) {
-						tuple.setField(Float.parseFloat((String) o), i);
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case DOUBLE:
-					if(o instanceof String) {
-						tuple.setField(Double.parseDouble((String) o), i);
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case STRING:
-					tuple.setField(o, i);
-					break;
-				case BINARY:
-					if(o instanceof String) {
-						throw new RuntimeException("Cannot handle partition keys of type BINARY.");
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case ARRAY:
-					if(o instanceof String) {
-						throw new RuntimeException("Cannot handle partition keys of type ARRAY.");
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case MAP:
-					if(o instanceof String) {
-						throw new RuntimeException("Cannot handle partition keys of type MAP.");
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				case STRUCT:
-					if(o instanceof String) {
-						throw new RuntimeException("Cannot handle partition keys of type STRUCT.");
-					} else {
-						tuple.setField(o, i);
-					}
-					break;
-				default:
-					throw new RuntimeException("Invalid Type");
-			}
-		}
-
-		return (T)tuple;
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hcatalog/src/main/scala/org/apache/flink/hcatalog/scala/HCatInputFormat.scala
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hcatalog/src/main/scala/org/apache/flink/hcatalog/scala/HCatInputFormat.scala b/flink-batch-connectors/flink-hcatalog/src/main/scala/org/apache/flink/hcatalog/scala/HCatInputFormat.scala
deleted file mode 100644
index 0299ee1..0000000
--- a/flink-batch-connectors/flink-hcatalog/src/main/scala/org/apache/flink/hcatalog/scala/HCatInputFormat.scala
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hcatalog.scala
-
-import org.apache.flink.configuration
-import org.apache.flink.hcatalog.HCatInputFormatBase
-import org.apache.hadoop.conf.Configuration
-import org.apache.hive.hcatalog.data.HCatRecord
-import org.apache.hive.hcatalog.data.schema.HCatFieldSchema
-
-/**
- * A InputFormat to read from HCatalog tables.
- * The InputFormat supports projection (selection and order of fields) and partition filters.
- *
- * Data can be returned as [[HCatRecord]] or Scala tuples.
- * Scala tuples support only up to 22 fields.
- *
- */
-class HCatInputFormat[T](
-                        database: String,
-                        table: String,
-                        config: Configuration
-                          ) extends HCatInputFormatBase[T](database, table, config) {
-
-  def this(database: String, table: String) {
-    this(database, table, new Configuration)
-  }
-
-  var vals: Array[Any] = Array[Any]()
-
-  override def configure(parameters: configuration.Configuration): Unit = {
-    super.configure(parameters)
-    vals = new Array[Any](fieldNames.length)
-  }
-
-  override protected def getMaxFlinkTupleSize: Int = 22
-
-  override protected def buildFlinkTuple(t: T, record: HCatRecord): T = {
-
-    // Extract all fields from HCatRecord
-    var i: Int = 0
-    while (i < this.fieldNames.length) {
-
-        val o: AnyRef = record.get(this.fieldNames(i), this.outputSchema)
-
-        // partition columns are returned as String
-        //   Check and convert to actual type.
-        this.outputSchema.get(i).getType match {
-          case HCatFieldSchema.Type.INT =>
-            if (o.isInstanceOf[String]) {
-              vals(i) = o.asInstanceOf[String].toInt
-            }
-            else {
-              vals(i) = o.asInstanceOf[Int]
-            }
-          case HCatFieldSchema.Type.TINYINT =>
-            if (o.isInstanceOf[String]) {
-              vals(i) = o.asInstanceOf[String].toInt.toByte
-            }
-            else {
-              vals(i) = o.asInstanceOf[Byte]
-            }
-          case HCatFieldSchema.Type.SMALLINT =>
-            if (o.isInstanceOf[String]) {
-              vals(i) = o.asInstanceOf[String].toInt.toShort
-            }
-            else {
-              vals(i) = o.asInstanceOf[Short]
-            }
-          case HCatFieldSchema.Type.BIGINT =>
-            if (o.isInstanceOf[String]) {
-              vals(i) = o.asInstanceOf[String].toLong
-            }
-            else {
-              vals(i) = o.asInstanceOf[Long]
-            }
-          case HCatFieldSchema.Type.BOOLEAN =>
-            if (o.isInstanceOf[String]) {
-              vals(i) = o.asInstanceOf[String].toBoolean
-            }
-            else {
-              vals(i) = o.asInstanceOf[Boolean]
-            }
-          case HCatFieldSchema.Type.FLOAT =>
-            if (o.isInstanceOf[String]) {
-              vals(i) = o.asInstanceOf[String].toFloat
-            }
-            else {
-              vals(i) = o.asInstanceOf[Float]
-            }
-          case HCatFieldSchema.Type.DOUBLE =>
-            if (o.isInstanceOf[String]) {
-              vals(i) = o.asInstanceOf[String].toDouble
-            }
-            else {
-              vals(i) = o.asInstanceOf[Double]
-            }
-          case HCatFieldSchema.Type.STRING =>
-            vals(i) = o
-          case HCatFieldSchema.Type.BINARY =>
-            if (o.isInstanceOf[String]) {
-              throw new RuntimeException("Cannot handle partition keys of type BINARY.")
-            }
-            else {
-              vals(i) = o.asInstanceOf[Array[Byte]]
-            }
-          case HCatFieldSchema.Type.ARRAY =>
-            if (o.isInstanceOf[String]) {
-              throw new RuntimeException("Cannot handle partition keys of type ARRAY.")
-            }
-            else {
-              vals(i) = o.asInstanceOf[List[Object]]
-            }
-          case HCatFieldSchema.Type.MAP =>
-            if (o.isInstanceOf[String]) {
-              throw new RuntimeException("Cannot handle partition keys of type MAP.")
-            }
-            else {
-              vals(i) = o.asInstanceOf[Map[Object, Object]]
-            }
-          case HCatFieldSchema.Type.STRUCT =>
-            if (o.isInstanceOf[String]) {
-              throw new RuntimeException("Cannot handle partition keys of type STRUCT.")
-            }
-            else {
-              vals(i) = o.asInstanceOf[List[Object]]
-            }
-          case _ =>
-            throw new RuntimeException("Invalid type " + this.outputSchema.get(i).getType +
-              " encountered.")
-        }
-
-        i += 1
-      }
-    createScalaTuple(vals)
-  }
-
-  private def createScalaTuple(vals: Array[Any]): T = {
-
-    this.fieldNames.length match {
-      case 1 =>
-        new Tuple1(vals(0)).asInstanceOf[T]
-      case 2 =>
-        new Tuple2(vals(0), vals(1)).asInstanceOf[T]
-      case 3 =>
-        new Tuple3(vals(0), vals(1), vals(2)).asInstanceOf[T]
-      case 4 =>
-        new Tuple4(vals(0), vals(1), vals(2), vals(3)).asInstanceOf[T]
-      case 5 =>
-        new Tuple5(vals(0), vals(1), vals(2), vals(3), vals(4)).asInstanceOf[T]
-      case 6 =>
-        new Tuple6(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5)).asInstanceOf[T]
-      case 7 =>
-        new Tuple7(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6)).asInstanceOf[T]
-      case 8 =>
-        new Tuple8(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7))
-          .asInstanceOf[T]
-      case 9 =>
-        new Tuple9(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8)).asInstanceOf[T]
-      case 10 =>
-        new Tuple10(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9)).asInstanceOf[T]
-      case 11 =>
-        new Tuple11(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10)).asInstanceOf[T]
-      case 12 =>
-        new Tuple12(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11)).asInstanceOf[T]
-      case 13 =>
-        new Tuple13(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12)).asInstanceOf[T]
-      case 14 =>
-        new Tuple14(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13)).asInstanceOf[T]
-      case 15 =>
-        new Tuple15(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14)).asInstanceOf[T]
-      case 16 =>
-        new Tuple16(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15))
-          .asInstanceOf[T]
-      case 17 =>
-        new Tuple17(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
-          vals(16)).asInstanceOf[T]
-      case 18 =>
-        new Tuple18(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
-          vals(16), vals(17)).asInstanceOf[T]
-      case 19 =>
-        new Tuple19(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
-          vals(16), vals(17), vals(18)).asInstanceOf[T]
-      case 20 =>
-        new Tuple20(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
-          vals(16), vals(17), vals(18), vals(19)).asInstanceOf[T]
-      case 21 =>
-        new Tuple21(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
-          vals(16), vals(17), vals(18), vals(19), vals(20)).asInstanceOf[T]
-      case 22 =>
-        new Tuple22(vals(0), vals(1), vals(2), vals(3), vals(4), vals(5), vals(6), vals(7),
-          vals(8), vals(9), vals(10), vals(11), vals(12), vals(13), vals(14), vals(15),
-          vals(16), vals(17), vals(18), vals(19), vals(20), vals(21)).asInstanceOf[T]
-      case _ =>
-        throw new RuntimeException("Only up to 22 fields supported for Scala Tuples.")
-
-  }
-
-  }
-}

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


[31/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
new file mode 100644
index 0000000..2e06160
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.junit.Assert;
+import org.junit.Test;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class FlinkKafkaProducerBaseTest {
+
+	/**
+	 * Tests that the constructor eagerly checks bootstrap servers are set in config
+	 */
+	@Test(expected = IllegalArgumentException.class)
+	public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception {
+		// no bootstrap servers set in props
+		Properties props = new Properties();
+		// should throw IllegalArgumentException
+		new DummyFlinkKafkaProducer<>(props, null);
+	}
+
+	/**
+	 * Tests that constructor defaults to key value serializers in config to byte array deserializers if not set
+	 */
+	@Test
+	public void testKeyValueDeserializersSetIfMissing() throws Exception {
+		Properties props = new Properties();
+		props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:12345");
+		// should set missing key value deserializers
+		new DummyFlinkKafkaProducer<>(props, null);
+
+		assertTrue(props.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG));
+		assertTrue(props.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG));
+		assertTrue(props.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).equals(ByteArraySerializer.class.getCanonicalName()));
+		assertTrue(props.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).equals(ByteArraySerializer.class.getCanonicalName()));
+	}
+
+	/**
+	 * Tests that partitions list is determinate and correctly provided to custom partitioner
+	 */
+	@Test
+	public void testPartitionerOpenedWithDeterminatePartitionList() throws Exception {
+		KafkaPartitioner mockPartitioner = mock(KafkaPartitioner.class);
+		RuntimeContext mockRuntimeContext = mock(RuntimeContext.class);
+		when(mockRuntimeContext.getIndexOfThisSubtask()).thenReturn(0);
+		when(mockRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(1);
+
+		DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer(
+			FakeStandardProducerConfig.get(), mockPartitioner);
+		producer.setRuntimeContext(mockRuntimeContext);
+
+		producer.open(new Configuration());
+
+		// the internal mock KafkaProducer will return an out-of-order list of 4 partitions,
+		// which should be sorted before provided to the custom partitioner's open() method
+		int[] correctPartitionList = {0, 1, 2, 3};
+		verify(mockPartitioner).open(0, 1, correctPartitionList);
+	}
+
+	/**
+	 * Test ensuring that the producer is not dropping buffered records.;
+	 * we set a timeout because the test will not finish if the logic is broken
+	 */
+	@Test(timeout=5000)
+	public void testAtLeastOnceProducer() throws Throwable {
+		runAtLeastOnceTest(true);
+	}
+
+	/**
+	 * Ensures that the at least once producing test fails if the flushing is disabled
+	 */
+	@Test(expected = AssertionError.class, timeout=5000)
+	public void testAtLeastOnceProducerFailsIfFlushingDisabled() throws Throwable {
+		runAtLeastOnceTest(false);
+	}
+
+	private void runAtLeastOnceTest(boolean flushOnCheckpoint) throws Throwable {
+		final AtomicBoolean snapshottingFinished = new AtomicBoolean(false);
+		final DummyFlinkKafkaProducer<String> producer = new DummyFlinkKafkaProducer<>(
+			FakeStandardProducerConfig.get(), null, snapshottingFinished);
+		producer.setFlushOnCheckpoint(flushOnCheckpoint);
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness =
+				new OneInputStreamOperatorTestHarness<>(new StreamSink(producer));
+
+		testHarness.open();
+
+		for (int i = 0; i < 100; i++) {
+			testHarness.processElement(new StreamRecord<>("msg-" + i));
+		}
+
+		// start a thread confirming all pending records
+		final Tuple1<Throwable> runnableError = new Tuple1<>(null);
+		final Thread threadA = Thread.currentThread();
+
+		Runnable confirmer = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					MockProducer mp = producer.getProducerInstance();
+					List<Callback> pending = mp.getPending();
+
+					// we need to find out if the snapshot() method blocks forever
+					// this is not possible. If snapshot() is running, it will
+					// start removing elements from the pending list.
+					synchronized (threadA) {
+						threadA.wait(500L);
+					}
+					// we now check that no records have been confirmed yet
+					Assert.assertEquals(100, pending.size());
+					Assert.assertFalse("Snapshot method returned before all records were confirmed",
+						snapshottingFinished.get());
+
+					// now confirm all checkpoints
+					for (Callback c: pending) {
+						c.onCompletion(null, null);
+					}
+					pending.clear();
+				} catch(Throwable t) {
+					runnableError.f0 = t;
+				}
+			}
+		};
+		Thread threadB = new Thread(confirmer);
+		threadB.start();
+
+		// this should block:
+		testHarness.snapshot(0, 0);
+
+		synchronized (threadA) {
+			threadA.notifyAll(); // just in case, to let the test fail faster
+		}
+		Assert.assertEquals(0, producer.getProducerInstance().getPending().size());
+		Deadline deadline = FiniteDuration.apply(5, "s").fromNow();
+		while (deadline.hasTimeLeft() && threadB.isAlive()) {
+			threadB.join(500);
+		}
+		Assert.assertFalse("Thread A is expected to be finished at this point. If not, the test is prone to fail", threadB.isAlive());
+		if (runnableError.f0 != null) {
+			throw runnableError.f0;
+		}
+
+		testHarness.close();
+	}
+
+
+	// ------------------------------------------------------------------------
+
+	private static class DummyFlinkKafkaProducer<T> extends FlinkKafkaProducerBase<T> {
+		private static final long serialVersionUID = 1L;
+
+		private transient MockProducer prod;
+		private AtomicBoolean snapshottingFinished;
+
+		@SuppressWarnings("unchecked")
+		public DummyFlinkKafkaProducer(Properties producerConfig, KafkaPartitioner partitioner, AtomicBoolean snapshottingFinished) {
+			super("dummy-topic", (KeyedSerializationSchema< T >) mock(KeyedSerializationSchema.class), producerConfig, partitioner);
+			this.snapshottingFinished = snapshottingFinished;
+		}
+
+		// constructor variant for test irrelated to snapshotting
+		@SuppressWarnings("unchecked")
+		public DummyFlinkKafkaProducer(Properties producerConfig, KafkaPartitioner partitioner) {
+			super("dummy-topic", (KeyedSerializationSchema< T >) mock(KeyedSerializationSchema.class), producerConfig, partitioner);
+			this.snapshottingFinished = new AtomicBoolean(true);
+		}
+
+		@Override
+		protected <K, V> KafkaProducer<K, V> getKafkaProducer(Properties props) {
+			this.prod = new MockProducer();
+			return this.prod;
+		}
+
+		@Override
+		public void snapshotState(FunctionSnapshotContext ctx) throws Exception {
+			// call the actual snapshot state
+			super.snapshotState(ctx);
+			// notify test that snapshotting has been done
+			snapshottingFinished.set(true);
+		}
+
+		@Override
+		protected void flush() {
+			this.prod.flush();
+		}
+
+		public MockProducer getProducerInstance() {
+			return this.prod;
+		}
+	}
+
+	private static class MockProducer<K, V> extends KafkaProducer<K, V> {
+		List<Callback> pendingCallbacks = new ArrayList<>();
+
+		public MockProducer() {
+			super(FakeStandardProducerConfig.get());
+		}
+
+		@Override
+		public Future<RecordMetadata> send(ProducerRecord<K, V> record) {
+			throw new UnsupportedOperationException("Unexpected");
+		}
+
+		@Override
+		public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
+			pendingCallbacks.add(callback);
+			return null;
+		}
+
+		@Override
+		public List<PartitionInfo> partitionsFor(String topic) {
+			List<PartitionInfo> list = new ArrayList<>();
+			// deliberately return an out-of-order partition list
+			list.add(new PartitionInfo(topic, 3, null, null, null));
+			list.add(new PartitionInfo(topic, 1, null, null, null));
+			list.add(new PartitionInfo(topic, 0, null, null, null));
+			list.add(new PartitionInfo(topic, 2, null, null, null));
+			return list;
+		}
+
+		@Override
+		public Map<MetricName, ? extends Metric> metrics() {
+			return null;
+		}
+
+
+		public List<Callback> getPending() {
+			return this.pendingCallbacks;
+		}
+
+		public void flush() {
+			while (pendingCallbacks.size() > 0) {
+				try {
+					Thread.sleep(10);
+				} catch (InterruptedException e) {
+					throw new RuntimeException("Unable to flush producer, task was interrupted");
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java
new file mode 100644
index 0000000..1882a7e
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.streaming.util.serialization.JSONDeserializationSchema;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class JSONDeserializationSchemaTest {
+	@Test
+	public void testDeserialize() throws IOException {
+		ObjectMapper mapper = new ObjectMapper();
+		ObjectNode initialValue = mapper.createObjectNode();
+		initialValue.put("key", 4).put("value", "world");
+		byte[] serializedValue = mapper.writeValueAsBytes(initialValue);
+
+		JSONDeserializationSchema schema = new JSONDeserializationSchema();
+		ObjectNode deserializedValue = schema.deserialize(serializedValue);
+
+		Assert.assertEquals(4, deserializedValue.get("key").asInt());
+		Assert.assertEquals("world", deserializedValue.get("value").asText());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java
new file mode 100644
index 0000000..86d3105
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class JSONKeyValueDeserializationSchemaTest {
+	@Test
+	public void testDeserializeWithoutMetadata() throws IOException {
+		ObjectMapper mapper = new ObjectMapper();
+		ObjectNode initialKey = mapper.createObjectNode();
+		initialKey.put("index", 4);
+		byte[] serializedKey = mapper.writeValueAsBytes(initialKey);
+
+		ObjectNode initialValue = mapper.createObjectNode();
+		initialValue.put("word", "world");
+		byte[] serializedValue = mapper.writeValueAsBytes(initialValue);
+
+		JSONKeyValueDeserializationSchema schema = new JSONKeyValueDeserializationSchema(false);
+		ObjectNode deserializedValue = schema.deserialize(serializedKey, serializedValue, "", 0, 0);
+
+
+		Assert.assertTrue(deserializedValue.get("metadata") == null);
+		Assert.assertEquals(4, deserializedValue.get("key").get("index").asInt());
+		Assert.assertEquals("world", deserializedValue.get("value").get("word").asText());
+	}
+
+	@Test
+	public void testDeserializeWithMetadata() throws IOException {
+		ObjectMapper mapper = new ObjectMapper();
+		ObjectNode initialKey = mapper.createObjectNode();
+		initialKey.put("index", 4);
+		byte[] serializedKey = mapper.writeValueAsBytes(initialKey);
+
+		ObjectNode initialValue = mapper.createObjectNode();
+		initialValue.put("word", "world");
+		byte[] serializedValue = mapper.writeValueAsBytes(initialValue);
+
+		JSONKeyValueDeserializationSchema schema = new JSONKeyValueDeserializationSchema(true);
+		ObjectNode deserializedValue = schema.deserialize(serializedKey, serializedValue, "topic#1", 3, 4);
+
+		Assert.assertEquals(4, deserializedValue.get("key").get("index").asInt());
+		Assert.assertEquals("world", deserializedValue.get("value").get("word").asText());
+		Assert.assertEquals("topic#1", deserializedValue.get("metadata").get("topic").asText());
+		Assert.assertEquals(4, deserializedValue.get("metadata").get("offset").asInt());
+		Assert.assertEquals(3, deserializedValue.get("metadata").get("partition").asInt());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java
new file mode 100644
index 0000000..68225e2
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JsonRowDeserializationSchemaTest {
+
+	/**
+	 * Tests simple deserialization.
+	 */
+	@Test
+	public void testDeserialization() throws Exception {
+		long id = 1238123899121L;
+		String name = "asdlkjasjkdla998y1122";
+		byte[] bytes = new byte[1024];
+		ThreadLocalRandom.current().nextBytes(bytes);
+
+		ObjectMapper objectMapper = new ObjectMapper();
+
+		// Root
+		ObjectNode root = objectMapper.createObjectNode();
+		root.put("id", id);
+		root.put("name", name);
+		root.put("bytes", bytes);
+
+		byte[] serializedJson = objectMapper.writeValueAsBytes(root);
+
+		JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(
+				new String[] { "id", "name", "bytes" },
+				new Class<?>[] { Long.class, String.class, byte[].class });
+
+		Row deserialized = deserializationSchema.deserialize(serializedJson);
+
+		assertEquals(3, deserialized.productArity());
+		assertEquals(id, deserialized.productElement(0));
+		assertEquals(name, deserialized.productElement(1));
+		assertArrayEquals(bytes, (byte[]) deserialized.productElement(2));
+	}
+
+	/**
+	 * Tests deserialization with non-existing field name.
+	 */
+	@Test
+	public void testMissingNode() throws Exception {
+		ObjectMapper objectMapper = new ObjectMapper();
+
+		// Root
+		ObjectNode root = objectMapper.createObjectNode();
+		root.put("id", 123123123);
+		byte[] serializedJson = objectMapper.writeValueAsBytes(root);
+
+		JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(
+				new String[] { "name" },
+				new Class<?>[] { String.class });
+
+		Row row = deserializationSchema.deserialize(serializedJson);
+
+		assertEquals(1, row.productArity());
+		assertNull("Missing field not null", row.productElement(0));
+
+		deserializationSchema.setFailOnMissingField(true);
+
+		try {
+			deserializationSchema.deserialize(serializedJson);
+			fail("Did not throw expected Exception");
+		} catch (IOException e) {
+			assertTrue(e.getCause() instanceof IllegalStateException);
+		}
+	}
+
+	/**
+	 * Tests that number of field names and types has to match.
+	 */
+	@Test
+	public void testNumberOfFieldNamesAndTypesMismatch() throws Exception {
+		try {
+			new JsonRowDeserializationSchema(
+					new String[] { "one", "two", "three" },
+					new Class<?>[] { Long.class });
+			fail("Did not throw expected Exception");
+		} catch (IllegalArgumentException ignored) {
+			// Expected
+		}
+
+		try {
+			new JsonRowDeserializationSchema(
+					new String[] { "one" },
+					new Class<?>[] { Long.class, String.class });
+			fail("Did not throw expected Exception");
+		} catch (IllegalArgumentException ignored) {
+			// Expected
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java
new file mode 100644
index 0000000..92af15d
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class JsonRowSerializationSchemaTest {
+	@Test
+	public void testRowSerialization() throws IOException {
+		String[] fieldNames = new String[] {"f1", "f2", "f3"};
+		Class[] fieldTypes = new Class[] {Integer.class, Boolean.class, String.class};
+		Row row = new Row(3);
+		row.setField(0, 1);
+		row.setField(1, true);
+		row.setField(2, "str");
+
+		Row resultRow = serializeAndDeserialize(fieldNames, fieldTypes, row);
+		assertEqualRows(row, resultRow);
+	}
+
+	@Test
+	public void testSerializationOfTwoRows() throws IOException {
+		String[] fieldNames = new String[] {"f1", "f2", "f3"};
+		Class[] fieldTypes = new Class[] {Integer.class, Boolean.class, String.class};
+		Row row1 = new Row(3);
+		row1.setField(0, 1);
+		row1.setField(1, true);
+		row1.setField(2, "str");
+
+		JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(fieldNames);
+		JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(fieldNames, fieldTypes);
+
+		byte[] bytes = serializationSchema.serialize(row1);
+		assertEqualRows(row1, deserializationSchema.deserialize(bytes));
+
+		Row row2 = new Row(3);
+		row2.setField(0, 10);
+		row2.setField(1, false);
+		row2.setField(2, "newStr");
+
+		bytes = serializationSchema.serialize(row2);
+		assertEqualRows(row2, deserializationSchema.deserialize(bytes));
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void testInputValidation() {
+		new JsonRowSerializationSchema(null);
+	}
+
+	@Test(expected = IllegalStateException.class)
+	public void testSerializeRowWithInvalidNumberOfFields() {
+		String[] fieldNames = new String[] {"f1", "f2", "f3"};
+		Row row = new Row(1);
+		row.setField(0, 1);
+
+		JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(fieldNames);
+		serializationSchema.serialize(row);
+	}
+
+	private Row serializeAndDeserialize(String[] fieldNames, Class[] fieldTypes, Row row) throws IOException {
+		JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(fieldNames);
+		JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(fieldNames, fieldTypes);
+
+		byte[] bytes = serializationSchema.serialize(row);
+		return deserializationSchema.deserialize(bytes);
+	}
+
+	private void assertEqualRows(Row expectedRow, Row resultRow) {
+		assertEquals("Deserialized row should have expected number of fields",
+			expectedRow.productArity(), resultRow.productArity());
+		for (int i = 0; i < expectedRow.productArity(); i++) {
+			assertEquals(String.format("Field number %d should be as in the original row", i),
+				expectedRow.productElement(i), resultRow.productElement(i));
+		}
+	}
+
+}

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


[27/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
new file mode 100644
index 0000000..612a4a7
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.internals;
+
+import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import com.amazonaws.services.kinesis.model.Record;
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Thread that does the actual data pulling from AWS Kinesis shards. Each thread is in charge of one Kinesis shard only.
+ */
+public class ShardConsumer<T> implements Runnable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class);
+
+	private final KinesisDeserializationSchema<T> deserializer;
+
+	private final KinesisProxyInterface kinesis;
+
+	private final int subscribedShardStateIndex;
+
+	private final KinesisDataFetcher<T> fetcherRef;
+
+	private final KinesisStreamShard subscribedShard;
+
+	private final int maxNumberOfRecordsPerFetch;
+	private final long fetchIntervalMillis;
+
+	private SequenceNumber lastSequenceNum;
+
+	/**
+	 * Creates a shard consumer.
+	 *
+	 * @param fetcherRef reference to the owning fetcher
+	 * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
+	 * @param subscribedShard the shard this consumer is subscribed to
+	 * @param lastSequenceNum the sequence number in the shard to start consuming
+	 */
+	public ShardConsumer(KinesisDataFetcher<T> fetcherRef,
+						Integer subscribedShardStateIndex,
+						KinesisStreamShard subscribedShard,
+						SequenceNumber lastSequenceNum) {
+		this(fetcherRef,
+			subscribedShardStateIndex,
+			subscribedShard,
+			lastSequenceNum,
+			KinesisProxy.create(fetcherRef.getConsumerConfiguration()));
+	}
+
+	/** This constructor is exposed for testing purposes */
+	protected ShardConsumer(KinesisDataFetcher<T> fetcherRef,
+							Integer subscribedShardStateIndex,
+							KinesisStreamShard subscribedShard,
+							SequenceNumber lastSequenceNum,
+							KinesisProxyInterface kinesis) {
+		this.fetcherRef = checkNotNull(fetcherRef);
+		this.subscribedShardStateIndex = checkNotNull(subscribedShardStateIndex);
+		this.subscribedShard = checkNotNull(subscribedShard);
+		this.lastSequenceNum = checkNotNull(lastSequenceNum);
+		checkArgument(
+			!lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()),
+			"Should not start a ShardConsumer if the shard has already been completely read.");
+
+		this.deserializer = fetcherRef.getClonedDeserializationSchema();
+
+		Properties consumerConfig = fetcherRef.getConsumerConfiguration();
+		this.kinesis = kinesis;
+		this.maxNumberOfRecordsPerFetch = Integer.valueOf(consumerConfig.getProperty(
+			ConsumerConfigConstants.SHARD_GETRECORDS_MAX,
+			Integer.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX)));
+		this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty(
+			ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
+			Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)));
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public void run() {
+		String nextShardItr;
+
+		try {
+			// before infinitely looping, we set the initial nextShardItr appropriately
+
+			if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.get())) {
+				// if the shard is already closed, there will be no latest next record to get for this shard
+				if (subscribedShard.isClosed()) {
+					nextShardItr = null;
+				} else {
+					nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.LATEST.toString(), null);
+				}
+			} else if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get())) {
+				nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.TRIM_HORIZON.toString(), null);
+			} else if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
+				nextShardItr = null;
+			} else {
+				// we will be starting from an actual sequence number (due to restore from failure).
+				// if the last sequence number refers to an aggregated record, we need to clean up any dangling sub-records
+				// from the last aggregated record; otherwise, we can simply start iterating from the record right after.
+
+				if (lastSequenceNum.isAggregated()) {
+					String itrForLastAggregatedRecord =
+						kinesis.getShardIterator(subscribedShard, ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber());
+
+					// get only the last aggregated record
+					GetRecordsResult getRecordsResult = getRecords(itrForLastAggregatedRecord, 1);
+
+					List<UserRecord> fetchedRecords = deaggregateRecords(
+						getRecordsResult.getRecords(),
+						subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
+						subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
+
+					long lastSubSequenceNum = lastSequenceNum.getSubSequenceNumber();
+					for (UserRecord record : fetchedRecords) {
+						// we have found a dangling sub-record if it has a larger subsequence number
+						// than our last sequence number; if so, collect the record and update state
+						if (record.getSubSequenceNumber() > lastSubSequenceNum) {
+							deserializeRecordForCollectionAndUpdateState(record);
+						}
+					}
+
+					// set the nextShardItr so we can continue iterating in the next while loop
+					nextShardItr = getRecordsResult.getNextShardIterator();
+				} else {
+					// the last record was non-aggregated, so we can simply start from the next record
+					nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber());
+				}
+			}
+
+			while(isRunning()) {
+				if (nextShardItr == null) {
+					fetcherRef.updateState(subscribedShardStateIndex, SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get());
+
+					// we can close this consumer thread once we've reached the end of the subscribed shard
+					break;
+				} else {
+					if (fetchIntervalMillis != 0) {
+						Thread.sleep(fetchIntervalMillis);
+					}
+
+					GetRecordsResult getRecordsResult = getRecords(nextShardItr, maxNumberOfRecordsPerFetch);
+
+					// each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding
+					List<UserRecord> fetchedRecords = deaggregateRecords(
+						getRecordsResult.getRecords(),
+						subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
+						subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
+
+					for (UserRecord record : fetchedRecords) {
+						deserializeRecordForCollectionAndUpdateState(record);
+					}
+
+					nextShardItr = getRecordsResult.getNextShardIterator();
+				}
+			}
+		} catch (Throwable t) {
+			fetcherRef.stopWithError(t);
+		}
+	}
+
+	/**
+	 * The loop in run() checks this before fetching next batch of records. Since this runnable will be executed
+	 * by the ExecutorService {@link KinesisDataFetcher#shardConsumersExecutor}, the only way to close down this thread
+	 * would be by calling shutdownNow() on {@link KinesisDataFetcher#shardConsumersExecutor} and let the executor service
+	 * interrupt all currently running {@link ShardConsumer}s.
+	 */
+	private boolean isRunning() {
+		return !Thread.interrupted();
+	}
+
+	/**
+	 * Deserializes a record for collection, and accordingly updates the shard state in the fetcher. The last
+	 * successfully collected sequence number in this shard consumer is also updated so that
+	 * {@link ShardConsumer#getRecords(String, int)} may be able to use the correct sequence number to refresh shard
+	 * iterators if necessary.
+	 *
+	 * Note that the server-side Kinesis timestamp is attached to the record when collected. When the
+	 * user programs uses {@link TimeCharacteristic#EventTime}, this timestamp will be used by default.
+	 *
+	 * @param record record to deserialize and collect
+	 * @throws IOException
+	 */
+	private void deserializeRecordForCollectionAndUpdateState(UserRecord record)
+		throws IOException {
+		ByteBuffer recordData = record.getData();
+
+		byte[] dataBytes = new byte[recordData.remaining()];
+		recordData.get(dataBytes);
+
+		final long approxArrivalTimestamp = record.getApproximateArrivalTimestamp().getTime();
+
+		final T value = deserializer.deserialize(
+			dataBytes,
+			record.getPartitionKey(),
+			record.getSequenceNumber(),
+			approxArrivalTimestamp,
+			subscribedShard.getStreamName(),
+			subscribedShard.getShard().getShardId());
+
+		SequenceNumber collectedSequenceNumber = (record.isAggregated())
+			? new SequenceNumber(record.getSequenceNumber(), record.getSubSequenceNumber())
+			: new SequenceNumber(record.getSequenceNumber());
+
+		fetcherRef.emitRecordAndUpdateState(
+			value,
+			approxArrivalTimestamp,
+			subscribedShardStateIndex,
+			collectedSequenceNumber);
+
+		lastSequenceNum = collectedSequenceNumber;
+	}
+
+	/**
+	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
+	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
+	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
+	 * be used for the next call to this method.
+	 *
+	 * Note: it is important that this method is not called again before all the records from the last result have been
+	 * fully collected with {@link ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
+	 * {@link ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
+	 * incorrect shard iteration if the iterator had to be refreshed.
+	 *
+	 * @param shardItr shard iterator to use
+	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
+	 * @return get records result
+	 * @throws InterruptedException
+	 */
+	private GetRecordsResult getRecords(String shardItr, int maxNumberOfRecords) throws InterruptedException {
+		GetRecordsResult getRecordsResult = null;
+		while (getRecordsResult == null) {
+			try {
+				getRecordsResult = kinesis.getRecords(shardItr, maxNumberOfRecords);
+			} catch (ExpiredIteratorException eiEx) {
+				LOG.warn("Encountered an unexpected expired iterator {} for shard {};" +
+					" refreshing the iterator ...", shardItr, subscribedShard);
+				shardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber());
+
+				// sleep for the fetch interval before the next getRecords attempt with the refreshed iterator
+				if (fetchIntervalMillis != 0) {
+					Thread.sleep(fetchIntervalMillis);
+				}
+			}
+		}
+		return getRecordsResult;
+	}
+
+	@SuppressWarnings("unchecked")
+	protected static List<UserRecord> deaggregateRecords(List<Record> records, String startingHashKey, String endingHashKey) {
+		return UserRecord.deaggregate(records, new BigInteger(startingHashKey), new BigInteger(endingHashKey));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java
new file mode 100644
index 0000000..53ed11b
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.model;
+
+import com.amazonaws.services.kinesis.model.Shard;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A serializable representation of a AWS Kinesis Stream shard. It is basically a wrapper class around the information
+ * provided along with {@link com.amazonaws.services.kinesis.model.Shard}, with some extra utility methods to
+ * determine whether or not a shard is closed and whether or not the shard is a result of parent shard splits or merges.
+ */
+public class KinesisStreamShard implements Serializable {
+
+	private static final long serialVersionUID = -6004217801761077536L;
+
+	private final String streamName;
+	private final Shard shard;
+
+	private final int cachedHash;
+
+	/**
+	 * Create a new KinesisStreamShard
+	 *
+	 * @param streamName
+	 *           the name of the Kinesis stream that this shard belongs to
+	 * @param shard
+	 *           the actual AWS Shard instance that will be wrapped within this KinesisStreamShard
+	 */
+	public KinesisStreamShard(String streamName, Shard shard) {
+		this.streamName = checkNotNull(streamName);
+		this.shard = checkNotNull(shard);
+
+		// since our description of Kinesis Streams shards can be fully defined with the stream name and shard id,
+		// our hash doesn't need to use hash code of Amazon's description of Shards, which uses other info for calculation
+		int hash = 17;
+		hash = 37 * hash + streamName.hashCode();
+		hash = 37 * hash + shard.getShardId().hashCode();
+		this.cachedHash = hash;
+	}
+
+	public String getStreamName() {
+		return streamName;
+	}
+
+	public boolean isClosed() {
+		return (shard.getSequenceNumberRange().getEndingSequenceNumber() != null);
+	}
+
+	public Shard getShard() {
+		return shard;
+	}
+
+	@Override
+	public String toString() {
+		return "KinesisStreamShard{" +
+			"streamName='" + streamName + "'" +
+			", shard='" + shard.toString() + "'}";
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (!(obj instanceof KinesisStreamShard)) {
+			return false;
+		}
+
+		if (obj == this) {
+			return true;
+		}
+
+		KinesisStreamShard other = (KinesisStreamShard) obj;
+
+		return streamName.equals(other.getStreamName()) && shard.equals(other.getShard());
+	}
+
+	@Override
+	public int hashCode() {
+		return cachedHash;
+	}
+
+	/**
+	 * Utility function to compare two shard ids
+	 *
+	 * @param firstShardId first shard id to compare
+	 * @param secondShardId second shard id to compare
+	 * @return a value less than 0 if the first shard id is smaller than the second shard id,
+	 *         or a value larger than 0 the first shard is larger then the second shard id,
+	 *         or 0 if they are equal
+	 */
+	public static int compareShardIds(String firstShardId, String secondShardId) {
+		if (!isValidShardId(firstShardId)) {
+			throw new IllegalArgumentException("The first shard id has invalid format.");
+		}
+
+		if (!isValidShardId(secondShardId)) {
+			throw new IllegalArgumentException("The second shard id has invalid format.");
+		}
+
+		// digit segment of the shard id starts at index 8
+		return Long.compare(Long.parseLong(firstShardId.substring(8)), Long.parseLong(secondShardId.substring(8)));
+	}
+
+	/**
+	 * Checks if a shard id has valid format.
+	 * Kinesis stream shard ids have 12-digit numbers left-padded with 0's,
+	 * prefixed with "shardId-", ex. "shardId-000000000015".
+	 *
+	 * @param shardId the shard id to check
+	 * @return whether the shard id is valid
+	 */
+	public static boolean isValidShardId(String shardId) {
+		if (shardId == null) { return false; }
+		return shardId.matches("^shardId-\\d{12}");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java
new file mode 100644
index 0000000..00181da
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.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.kinesis.model;
+
+/**
+ * A wrapper class that bundles a {@link KinesisStreamShard} with its last processed sequence number.
+ */
+public class KinesisStreamShardState {
+
+	private KinesisStreamShard kinesisStreamShard;
+	private SequenceNumber lastProcessedSequenceNum;
+
+	public KinesisStreamShardState(KinesisStreamShard kinesisStreamShard, SequenceNumber lastProcessedSequenceNum) {
+		this.kinesisStreamShard = kinesisStreamShard;
+		this.lastProcessedSequenceNum = lastProcessedSequenceNum;
+	}
+
+	public KinesisStreamShard getKinesisStreamShard() {
+		return this.kinesisStreamShard;
+	}
+
+	public SequenceNumber getLastProcessedSequenceNum() {
+		return this.lastProcessedSequenceNum;
+	}
+
+	public void setLastProcessedSequenceNum(SequenceNumber update) {
+		this.lastProcessedSequenceNum = update;
+	}
+
+	@Override
+	public String toString() {
+		return "KinesisStreamShardState{" +
+			"kinesisStreamShard='" + kinesisStreamShard.toString() + "'" +
+			", lastProcessedSequenceNumber='" + lastProcessedSequenceNum.toString() + "'}";
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (!(obj instanceof KinesisStreamShardState)) {
+			return false;
+		}
+
+		if (obj == this) {
+			return true;
+		}
+
+		KinesisStreamShardState other = (KinesisStreamShardState) obj;
+
+		return kinesisStreamShard.equals(other.getKinesisStreamShard()) && lastProcessedSequenceNum.equals(other.getLastProcessedSequenceNum());
+	}
+
+	@Override
+	public int hashCode() {
+		return 37 * (kinesisStreamShard.hashCode() + lastProcessedSequenceNum.hashCode());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java
new file mode 100644
index 0000000..8182201
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.model;
+
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
+
+/**
+ * Special flag values for sequence numbers in shards to indicate special positions.
+ * The value is initially set by {@link FlinkKinesisConsumer} when {@link KinesisDataFetcher}s are created.
+ * The KinesisDataFetchers will use this value to determine how to retrieve the starting shard iterator from AWS Kinesis.
+ */
+public enum SentinelSequenceNumber {
+
+	/** Flag value for shard's sequence numbers to indicate that the
+	 * shard should start to be read from the latest incoming records */
+	SENTINEL_LATEST_SEQUENCE_NUM( new SequenceNumber("LATEST_SEQUENCE_NUM") ),
+
+	/** Flag value for shard's sequence numbers to indicate that the shard should
+	 * start to be read from the earliest records that haven't expired yet */
+	SENTINEL_EARLIEST_SEQUENCE_NUM( new SequenceNumber("EARLIEST_SEQUENCE_NUM") ),
+
+	/** Flag value to indicate that we have already read the last record of this shard
+	 * (Note: Kinesis shards that have been closed due to a split or merge will have an ending data record) */
+	SENTINEL_SHARD_ENDING_SEQUENCE_NUM( new SequenceNumber("SHARD_ENDING_SEQUENCE_NUM") );
+
+	private SequenceNumber sentinel;
+
+	SentinelSequenceNumber(SequenceNumber sentinel) {
+		this.sentinel = sentinel;
+	}
+
+	public SequenceNumber get() {
+		return sentinel;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java
new file mode 100644
index 0000000..021f53f
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.model;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A serializable representation of a Kinesis record's sequence number. It has two fields: the main sequence number,
+ * and also a subsequence number. If this {@link SequenceNumber} is referring to an aggregated Kinesis record, the
+ * subsequence number will be a non-negative value representing the order of the sub-record within the aggregation.
+ */
+public class SequenceNumber implements Serializable {
+
+	private static final long serialVersionUID = 876972197938972667L;
+
+	private static final String DELIMITER = "-";
+
+	private final String sequenceNumber;
+	private final long subSequenceNumber;
+
+	private final int cachedHash;
+
+	/**
+	 * Create a new instance for a non-aggregated Kinesis record without a subsequence number.
+	 * @param sequenceNumber the sequence number
+	 */
+	public SequenceNumber(String sequenceNumber) {
+		this(sequenceNumber, -1);
+	}
+
+	/**
+	 * Create a new instance, with the specified sequence number and subsequence number.
+	 * To represent the sequence number for a non-aggregated Kinesis record, the subsequence number should be -1.
+	 * Otherwise, give a non-negative sequence number to represent an aggregated Kinesis record.
+	 *
+	 * @param sequenceNumber the sequence number
+	 * @param subSequenceNumber the subsequence number (-1 to represent non-aggregated Kinesis records)
+	 */
+	public SequenceNumber(String sequenceNumber, long subSequenceNumber) {
+		this.sequenceNumber = checkNotNull(sequenceNumber);
+		this.subSequenceNumber = subSequenceNumber;
+
+		this.cachedHash = 37 * (sequenceNumber.hashCode() + Long.valueOf(subSequenceNumber).hashCode());
+	}
+
+	public boolean isAggregated() {
+		return subSequenceNumber >= 0;
+	}
+
+	public String getSequenceNumber() {
+		return sequenceNumber;
+	}
+
+	public long getSubSequenceNumber() {
+		return subSequenceNumber;
+	}
+
+	@Override
+	public String toString() {
+		if (isAggregated()) {
+			return sequenceNumber + DELIMITER + subSequenceNumber;
+		} else {
+			return sequenceNumber;
+		}
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (!(obj instanceof SequenceNumber)) {
+			return false;
+		}
+
+		if (obj == this) {
+			return true;
+		}
+
+		SequenceNumber other = (SequenceNumber) obj;
+
+		return sequenceNumber.equals(other.getSequenceNumber())
+			&& (subSequenceNumber == other.getSubSequenceNumber());
+	}
+
+	@Override
+	public int hashCode() {
+		return cachedHash;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java
new file mode 100644
index 0000000..04b1654
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.proxy;
+
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Basic model class to bundle the shards retrieved from Kinesis on a {@link KinesisProxyInterface#getShardList(Map)} call.
+ */
+public class GetShardListResult {
+
+	private final Map<String, LinkedList<KinesisStreamShard>> streamsToRetrievedShardList = new HashMap<>();
+
+	public void addRetrievedShardToStream(String stream, KinesisStreamShard retrievedShard) {
+		if (!streamsToRetrievedShardList.containsKey(stream)) {
+			streamsToRetrievedShardList.put(stream, new LinkedList<KinesisStreamShard>());
+		}
+		streamsToRetrievedShardList.get(stream).add(retrievedShard);
+	}
+
+	public void addRetrievedShardsToStream(String stream, List<KinesisStreamShard> retrievedShards) {
+		if (retrievedShards.size() != 0) {
+			if (!streamsToRetrievedShardList.containsKey(stream)) {
+				streamsToRetrievedShardList.put(stream, new LinkedList<KinesisStreamShard>());
+			}
+			streamsToRetrievedShardList.get(stream).addAll(retrievedShards);
+		}
+	}
+
+	public List<KinesisStreamShard> getRetrievedShardListOfStream(String stream) {
+		if (!streamsToRetrievedShardList.containsKey(stream)) {
+			return null;
+		} else {
+			return streamsToRetrievedShardList.get(stream);
+		}
+	}
+
+	public KinesisStreamShard getLastSeenShardOfStream(String stream) {
+		if (!streamsToRetrievedShardList.containsKey(stream)) {
+			return null;
+		} else {
+			return streamsToRetrievedShardList.get(stream).getLast();
+		}
+	}
+
+	public boolean hasRetrievedShards() {
+		return !streamsToRetrievedShardList.isEmpty();
+	}
+
+	public Set<String> getStreamsWithRetrievedShards() {
+		return streamsToRetrievedShardList.keySet();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java
new file mode 100644
index 0000000..9ffc8e6
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.proxy;
+
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.model.DescribeStreamRequest;
+import com.amazonaws.services.kinesis.model.DescribeStreamResult;
+import com.amazonaws.services.kinesis.model.GetRecordsRequest;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import com.amazonaws.services.kinesis.model.GetShardIteratorResult;
+import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException;
+import com.amazonaws.services.kinesis.model.LimitExceededException;
+import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
+import com.amazonaws.services.kinesis.model.StreamStatus;
+import com.amazonaws.services.kinesis.model.Shard;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Map;
+import java.util.Random;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Kinesis proxy implementation - a utility class that is used as a proxy to make
+ * calls to AWS Kinesis for several functions, such as getting a list of shards and
+ * fetching a batch of data records starting from a specified record sequence number.
+ *
+ * NOTE:
+ * In the AWS KCL library, there is a similar implementation - {@link com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxy}.
+ * This implementation differs mainly in that we can make operations to arbitrary Kinesis streams, which is a needed
+ * functionality for the Flink Kinesis Connecter since the consumer may simultaneously read from multiple Kinesis streams.
+ */
+public class KinesisProxy implements KinesisProxyInterface {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KinesisProxy.class);
+
+	/** The actual Kinesis client from the AWS SDK that we will be using to make calls */
+	private final AmazonKinesisClient kinesisClient;
+
+	/** Random seed used to calculate backoff jitter for Kinesis operations */
+	private final static Random seed = new Random();
+
+	// ------------------------------------------------------------------------
+	//  describeStream() related performance settings
+	// ------------------------------------------------------------------------
+
+	/** Base backoff millis for the describe stream operation */
+	private final long describeStreamBaseBackoffMillis;
+
+	/** Maximum backoff millis for the describe stream operation */
+	private final long describeStreamMaxBackoffMillis;
+
+	/** Exponential backoff power constant for the describe stream operation */
+	private final double describeStreamExpConstant;
+
+	// ------------------------------------------------------------------------
+	//  getRecords() related performance settings
+	// ------------------------------------------------------------------------
+
+	/** Base backoff millis for the get records operation */
+	private final long getRecordsBaseBackoffMillis;
+
+	/** Maximum backoff millis for the get records operation */
+	private final long getRecordsMaxBackoffMillis;
+
+	/** Exponential backoff power constant for the get records operation */
+	private final double getRecordsExpConstant;
+
+	/** Maximum attempts for the get records operation */
+	private final int getRecordsMaxAttempts;
+
+	// ------------------------------------------------------------------------
+	//  getShardIterator() related performance settings
+	// ------------------------------------------------------------------------
+
+	/** Base backoff millis for the get shard iterator operation */
+	private final long getShardIteratorBaseBackoffMillis;
+
+	/** Maximum backoff millis for the get shard iterator operation */
+	private final long getShardIteratorMaxBackoffMillis;
+
+	/** Exponential backoff power constant for the get shard iterator operation */
+	private final double getShardIteratorExpConstant;
+
+	/** Maximum attempts for the get shard iterator operation */
+	private final int getShardIteratorMaxAttempts;
+
+	/**
+	 * Create a new KinesisProxy based on the supplied configuration properties
+	 *
+	 * @param configProps configuration properties containing AWS credential and AWS region info
+	 */
+	private KinesisProxy(Properties configProps) {
+		checkNotNull(configProps);
+
+		this.kinesisClient = AWSUtil.createKinesisClient(configProps);
+
+		this.describeStreamBaseBackoffMillis = Long.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE,
+				Long.toString(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE)));
+		this.describeStreamMaxBackoffMillis = Long.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX,
+				Long.toString(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX)));
+		this.describeStreamExpConstant = Double.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT,
+				Double.toString(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT)));
+
+		this.getRecordsBaseBackoffMillis = Long.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_BASE,
+				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE)));
+		this.getRecordsMaxBackoffMillis = Long.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_MAX,
+				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX)));
+		this.getRecordsExpConstant = Double.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT,
+				Double.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT)));
+		this.getRecordsMaxAttempts = Integer.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.SHARD_GETRECORDS_RETRIES,
+				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_RETRIES)));
+
+		this.getShardIteratorBaseBackoffMillis = Long.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_BASE,
+				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE)));
+		this.getShardIteratorMaxBackoffMillis = Long.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_MAX,
+				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX)));
+		this.getShardIteratorExpConstant = Double.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT,
+				Double.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT)));
+		this.getShardIteratorMaxAttempts = Integer.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.SHARD_GETITERATOR_RETRIES,
+				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETITERATOR_RETRIES)));
+
+	}
+
+	/**
+	 * Creates a Kinesis proxy.
+	 *
+	 * @param configProps configuration properties
+	 * @return the created kinesis proxy
+	 */
+	public static KinesisProxyInterface create(Properties configProps) {
+		return new KinesisProxy(configProps);
+	}
+
+	/**
+	 * {@inheritDoc}
+	 */
+	@Override
+	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException {
+		final GetRecordsRequest getRecordsRequest = new GetRecordsRequest();
+		getRecordsRequest.setShardIterator(shardIterator);
+		getRecordsRequest.setLimit(maxRecordsToGet);
+
+		GetRecordsResult getRecordsResult = null;
+
+		int attempt = 0;
+		while (attempt <= getRecordsMaxAttempts && getRecordsResult == null) {
+			try {
+				getRecordsResult = kinesisClient.getRecords(getRecordsRequest);
+			} catch (ProvisionedThroughputExceededException ex) {
+				long backoffMillis = fullJitterBackoff(
+					getRecordsBaseBackoffMillis, getRecordsMaxBackoffMillis, getRecordsExpConstant, attempt++);
+				LOG.warn("Got ProvisionedThroughputExceededException. Backing off for "
+					+ backoffMillis + " millis.");
+				Thread.sleep(backoffMillis);
+			}
+		}
+
+		if (getRecordsResult == null) {
+			throw new RuntimeException("Rate Exceeded for getRecords operation - all " + getRecordsMaxAttempts +
+				" retry attempts returned ProvisionedThroughputExceededException.");
+		}
+
+		return getRecordsResult;
+	}
+
+	/**
+	 * {@inheritDoc}
+	 */
+	@Override
+	public GetShardListResult getShardList(Map<String, String> streamNamesWithLastSeenShardIds) throws InterruptedException {
+		GetShardListResult result = new GetShardListResult();
+
+		for (Map.Entry<String,String> streamNameWithLastSeenShardId : streamNamesWithLastSeenShardIds.entrySet()) {
+			String stream = streamNameWithLastSeenShardId.getKey();
+			String lastSeenShardId = streamNameWithLastSeenShardId.getValue();
+			result.addRetrievedShardsToStream(stream, getShardsOfStream(stream, lastSeenShardId));
+		}
+		return result;
+	}
+
+	/**
+	 * {@inheritDoc}
+	 */
+	@Override
+	public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, @Nullable String startingSeqNum) throws InterruptedException {
+		GetShardIteratorResult getShardIteratorResult = null;
+
+		int attempt = 0;
+		while (attempt <= getShardIteratorMaxAttempts && getShardIteratorResult == null) {
+			try {
+				getShardIteratorResult =
+					kinesisClient.getShardIterator(shard.getStreamName(), shard.getShard().getShardId(), shardIteratorType, startingSeqNum);
+			} catch (ProvisionedThroughputExceededException ex) {
+				long backoffMillis = fullJitterBackoff(
+					getShardIteratorBaseBackoffMillis, getShardIteratorMaxBackoffMillis, getShardIteratorExpConstant, attempt++);
+				LOG.warn("Got ProvisionedThroughputExceededException. Backing off for "
+					+ backoffMillis + " millis.");
+				Thread.sleep(backoffMillis);
+			}
+		}
+
+		if (getShardIteratorResult == null) {
+			throw new RuntimeException("Rate Exceeded for getShardIterator operation - all " + getShardIteratorMaxAttempts +
+				" retry attempts returned ProvisionedThroughputExceededException.");
+		}
+		return getShardIteratorResult.getShardIterator();
+	}
+
+	private List<KinesisStreamShard> getShardsOfStream(String streamName, @Nullable String lastSeenShardId) throws InterruptedException {
+		List<KinesisStreamShard> shardsOfStream = new ArrayList<>();
+
+		DescribeStreamResult describeStreamResult;
+		do {
+			describeStreamResult = describeStream(streamName, lastSeenShardId);
+
+			List<Shard> shards = describeStreamResult.getStreamDescription().getShards();
+			for (Shard shard : shards) {
+				shardsOfStream.add(new KinesisStreamShard(streamName, shard));
+			}
+
+			if (shards.size() != 0) {
+				lastSeenShardId = shards.get(shards.size() - 1).getShardId();
+			}
+		} while (describeStreamResult.getStreamDescription().isHasMoreShards());
+
+		return shardsOfStream;
+	}
+
+	/**
+	 * Get metainfo for a Kinesis stream, which contains information about which shards this Kinesis stream possess.
+	 *
+	 * This method is using a "full jitter" approach described in AWS's article,
+	 * <a href="https://www.awsarchitectureblog.com/2015/03/backoff.html">"Exponential Backoff and Jitter"</a>.
+	 * This is necessary because concurrent calls will be made by all parallel subtask's fetcher. This
+	 * jitter backoff approach will help distribute calls across the fetchers over time.
+	 *
+	 * @param streamName the stream to describe
+	 * @param startShardId which shard to start with for this describe operation (earlier shard's infos will not appear in result)
+	 * @return the result of the describe stream operation
+	 */
+	private DescribeStreamResult describeStream(String streamName, @Nullable String startShardId) throws InterruptedException {
+		final DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest();
+		describeStreamRequest.setStreamName(streamName);
+		describeStreamRequest.setExclusiveStartShardId(startShardId);
+
+		DescribeStreamResult describeStreamResult = null;
+
+		// Call DescribeStream, with full-jitter backoff (if we get LimitExceededException).
+		int attemptCount = 0;
+		while (describeStreamResult == null) { // retry until we get a result
+			try {
+				describeStreamResult = kinesisClient.describeStream(describeStreamRequest);
+			} catch (LimitExceededException le) {
+				long backoffMillis = fullJitterBackoff(
+					describeStreamBaseBackoffMillis, describeStreamMaxBackoffMillis, describeStreamExpConstant, attemptCount++);
+				LOG.warn("Got LimitExceededException when describing stream " + streamName + ". Backing off for "
+					+ backoffMillis + " millis.");
+				Thread.sleep(backoffMillis);
+			} catch (ResourceNotFoundException re) {
+				throw new RuntimeException("Error while getting stream details", re);
+			}
+		}
+
+		String streamStatus = describeStreamResult.getStreamDescription().getStreamStatus();
+		if (!(streamStatus.equals(StreamStatus.ACTIVE.toString()) || streamStatus.equals(StreamStatus.UPDATING.toString()))) {
+			if (LOG.isWarnEnabled()) {
+				LOG.warn("The status of stream " + streamName + " is " + streamStatus + "; result of the current " +
+					"describeStream operation will not contain any shard information.");
+			}
+		}
+
+		// Kinesalite (mock implementation of Kinesis) does not correctly exclude shards before the exclusive
+		// start shard id in the returned shards list; check if we need to remove these erroneously returned shards
+		if (startShardId != null) {
+			List<Shard> shards = describeStreamResult.getStreamDescription().getShards();
+			Iterator<Shard> shardItr = shards.iterator();
+			while (shardItr.hasNext()) {
+				if (KinesisStreamShard.compareShardIds(shardItr.next().getShardId(), startShardId) <= 0) {
+					shardItr.remove();
+				}
+			}
+		}
+
+		return describeStreamResult;
+	}
+
+	private static long fullJitterBackoff(long base, long max, double power, int attempt) {
+		long exponentialBackoff = (long) Math.min(max, base * Math.pow(power, attempt));
+		return (long)(seed.nextDouble()*exponentialBackoff); // random jitter between 0 and the exponential backoff
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java
new file mode 100644
index 0000000..39ddc52
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.proxy;
+
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+
+import java.util.Map;
+
+/**
+ * Interface for a Kinesis proxy that operates on multiple Kinesis streams within the same AWS service region.
+ */
+public interface KinesisProxyInterface {
+
+	/**
+	 * Get a shard iterator from the specified position in a shard.
+	 * The retrieved shard iterator can be used in {@link KinesisProxyInterface#getRecords(String, int)}}
+	 * to read data from the Kinesis shard.
+	 *
+	 * @param shard the shard to get the iterator
+	 * @param shardIteratorType the iterator type, defining how the shard is to be iterated
+	 *                          (one of: TRIM_HORIZON, LATEST, AT_SEQUENCE_NUMBER, AFTER_SEQUENCE_NUMBER)
+	 * @param startingSeqNum sequence number, must be null if shardIteratorType is TRIM_HORIZON or LATEST
+	 * @return shard iterator which can be used to read data from Kinesis
+	 * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the
+	 *                              operation has exceeded the rate limit; this exception will be thrown
+	 *                              if the backoff is interrupted.
+	 */
+	String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) throws InterruptedException;
+
+	/**
+	 * Get the next batch of data records using a specific shard iterator
+	 *
+	 * @param shardIterator a shard iterator that encodes info about which shard to read and where to start reading
+	 * @param maxRecordsToGet the maximum amount of records to retrieve for this batch
+	 * @return the batch of retrieved records, also with a shard iterator that can be used to get the next batch
+	 * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the
+	 *                              operation has exceeded the rate limit; this exception will be thrown
+	 *                              if the backoff is interrupted.
+	 */
+	GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException;
+
+	/**
+	 * Get shard list of multiple Kinesis streams, ignoring the
+	 * shards of each stream before a specified last seen shard id.
+	 *
+	 * @param streamNamesWithLastSeenShardIds a map with stream as key, and last seen shard id as value
+	 * @return result of the shard list query
+	 * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the
+	 *                              operation has exceeded the rate limit; this exception will be thrown
+	 *                              if the backoff is interrupted.
+	 */
+	GetShardListResult getShardList(Map<String,String> streamNamesWithLastSeenShardIds) throws InterruptedException;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java
new file mode 100644
index 0000000..0effdd8
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.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.kinesis.serialization;
+
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * This is a deserialization schema specific for the Flink Kinesis Consumer. Different from the
+ * basic {@link DeserializationSchema}, this schema offers additional Kinesis-specific information
+ * about the record that may be useful to the user application.
+ *
+ * @param <T> The type created by the keyed deserialization schema.
+ */
+public interface KinesisDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
+
+	/**
+	 * Deserializes a Kinesis record's bytes
+	 *
+	 * @param recordValue the record's value as a byte array
+	 * @param partitionKey the record's partition key at the time of writing
+	 * @param seqNum the sequence number of this record in the Kinesis shard
+	 * @param approxArrivalTimestamp the server-side timestamp of when Kinesis received and stored the record
+	 * @param stream the name of the Kinesis stream that this record was sent to
+	 * @param shardId The identifier of the shard the record was sent to
+	 * @return the deserialized message as an Java object
+	 * @throws IOException
+	 */
+	T deserialize(byte[] recordValue, String partitionKey, String seqNum, long approxArrivalTimestamp, String stream, String shardId) throws IOException;
+
+	/**
+	 * Method to decide whether the element signals the end of the stream. If
+	 * true is returned the element won't be emitted.
+	 *
+	 * @param nextElement the element to test for the end-of-stream signal
+	 * @return true if the element signals end of stream, false otherwise
+	 */
+	// TODO FLINK-4194 ADD SUPPORT FOR boolean isEndOfStream(T nextElement);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java
new file mode 100644
index 0000000..6e66038
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.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.kinesis.serialization;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.io.IOException;
+
+/**
+ * A simple wrapper for using the {@link DeserializationSchema} with the {@link KinesisDeserializationSchema} interface
+ *
+ * @param <T> The type created by the deserialization schema.
+ */
+public class KinesisDeserializationSchemaWrapper<T> implements KinesisDeserializationSchema<T> {
+	private static final long serialVersionUID = 9143148962928375886L;
+
+	private final DeserializationSchema<T> deserializationSchema;
+
+	public KinesisDeserializationSchemaWrapper(DeserializationSchema<T> deserializationSchema) {
+		this.deserializationSchema = deserializationSchema;
+	}
+
+	@Override
+	public T deserialize(byte[] recordValue, String partitionKey, String seqNum, long approxArrivalTimestamp, String stream, String shardId)
+		throws IOException {
+		return deserializationSchema.deserialize(recordValue);
+	}
+
+	/*
+	FLINK-4194
+
+	@Override
+	public boolean isEndOfStream(T nextElement) {
+		return deserializationSchema.isEndOfStream(nextElement);
+	} */
+
+	@Override
+	public TypeInformation<T> getProducedType() {
+		return deserializationSchema.getProducedType();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java
new file mode 100644
index 0000000..03dd72c
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kinesis.serialization;
+
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+/**
+ * Kinesis-specific serialization schema, allowing users to specify a target stream based
+ * on a record's contents.
+ * @param <T>
+ */
+public interface KinesisSerializationSchema<T> extends Serializable {
+	/**
+	 * Serialize the given element into a ByteBuffer
+	 *
+	 * @param element The element to serialize
+	 * @return Serialized representation of the element
+	 */
+	ByteBuffer serialize(T element);
+
+	/**
+	 * Optional method to determine the target stream based on the element.
+	 * Return <code>null</code> to use the default stream
+	 *
+	 * @param element The element to determine the target stream from
+	 * @return target stream name
+	 */
+	String getTargetStream(T element);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java
new file mode 100644
index 0000000..cff69e5
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.util;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ClientConfigurationFactory;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.BasicAWSCredentials;
+import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
+import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
+import com.amazonaws.auth.SystemPropertiesCredentialsProvider;
+import com.amazonaws.auth.profile.ProfileCredentialsProvider;
+import com.amazonaws.regions.Region;
+import com.amazonaws.regions.Regions;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.CredentialProvider;
+
+import java.util.Properties;
+
+/**
+ * Some utilities specific to Amazon Web Service.
+ */
+public class AWSUtil {
+
+	/**
+	 * Creates an Amazon Kinesis Client.
+	 * @param configProps configuration properties containing the access key, secret key, and region
+	 * @return a new Amazon Kinesis Client
+	 */
+	public static AmazonKinesisClient createKinesisClient(Properties configProps) {
+		// set a Flink-specific user agent
+		ClientConfiguration awsClientConfig = new ClientConfigurationFactory().getConfig();
+		awsClientConfig.setUserAgent("Apache Flink " + EnvironmentInformation.getVersion() +
+			" (" + EnvironmentInformation.getRevisionInformation().commitId + ") Kinesis Connector");
+
+		// utilize automatic refreshment of credentials by directly passing the AWSCredentialsProvider
+		AmazonKinesisClient client = new AmazonKinesisClient(
+			AWSUtil.getCredentialsProvider(configProps), awsClientConfig);
+
+		client.setRegion(Region.getRegion(Regions.fromName(configProps.getProperty(AWSConfigConstants.AWS_REGION))));
+		if (configProps.containsKey(AWSConfigConstants.AWS_ENDPOINT)) {
+			client.setEndpoint(configProps.getProperty(AWSConfigConstants.AWS_ENDPOINT));
+		}
+		return client;
+	}
+
+	/**
+	 * Return a {@link AWSCredentialsProvider} instance corresponding to the configuration properties.
+	 *
+	 * @param configProps the configuration properties
+	 * @return The corresponding AWS Credentials Provider instance
+	 */
+	public static AWSCredentialsProvider getCredentialsProvider(final Properties configProps) {
+		CredentialProvider credentialProviderType = CredentialProvider.valueOf(configProps.getProperty(
+			AWSConfigConstants.AWS_CREDENTIALS_PROVIDER, CredentialProvider.BASIC.toString()));
+
+		AWSCredentialsProvider credentialsProvider;
+
+		switch (credentialProviderType) {
+			case ENV_VAR:
+				credentialsProvider = new EnvironmentVariableCredentialsProvider();
+				break;
+			case SYS_PROP:
+				credentialsProvider = new SystemPropertiesCredentialsProvider();
+				break;
+			case PROFILE:
+				String profileName = configProps.getProperty(
+					AWSConfigConstants.AWS_PROFILE_NAME, null);
+				String profileConfigPath = configProps.getProperty(
+					AWSConfigConstants.AWS_PROFILE_PATH, null);
+				credentialsProvider = (profileConfigPath == null)
+					? new ProfileCredentialsProvider(profileName)
+					: new ProfileCredentialsProvider(profileConfigPath, profileName);
+				break;
+			case AUTO:
+				credentialsProvider = new DefaultAWSCredentialsProviderChain();
+				break;
+			default:
+			case BASIC:
+				credentialsProvider = new AWSCredentialsProvider() {
+					@Override
+					public AWSCredentials getCredentials() {
+						return new BasicAWSCredentials(
+							configProps.getProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID),
+							configProps.getProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY));
+					}
+
+					@Override
+					public void refresh() {
+						// do nothing
+					}
+				};
+		}
+
+		return credentialsProvider;
+	}
+
+	/**
+	 * Checks whether or not a region ID is valid
+	 *
+	 * @param region The AWS region ID to check
+	 * @return true if the supplied region ID is valid, false otherwise
+	 */
+	public static boolean isValidRegion(String region) {
+		try {
+			Regions.fromName(region.toLowerCase());
+		} catch (IllegalArgumentException e) {
+			return false;
+		}
+		return true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
new file mode 100644
index 0000000..9aa14ad
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.util;
+
+import com.amazonaws.regions.Regions;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.CredentialProvider;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.InitialPosition;
+import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
+
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Utilities for Flink Kinesis connector configuration.
+ */
+public class KinesisConfigUtil {
+
+	/**
+	 * Validate configuration properties for {@link FlinkKinesisConsumer}.
+	 */
+	public static void validateConsumerConfiguration(Properties config) {
+		checkNotNull(config, "config can not be null");
+
+		validateAwsConfiguration(config);
+
+		if (config.containsKey(ConsumerConfigConstants.STREAM_INITIAL_POSITION)) {
+			String initPosType = config.getProperty(ConsumerConfigConstants.STREAM_INITIAL_POSITION);
+
+			// specified initial position in stream must be either LATEST or TRIM_HORIZON
+			try {
+				InitialPosition.valueOf(initPosType);
+			} catch (IllegalArgumentException e) {
+				StringBuilder sb = new StringBuilder();
+				for (InitialPosition pos : InitialPosition.values()) {
+					sb.append(pos.toString()).append(", ");
+				}
+				throw new IllegalArgumentException("Invalid initial position in stream set in config. Valid values are: " + sb.toString());
+			}
+		}
+
+		validateOptionalPositiveIntProperty(config, ConsumerConfigConstants.SHARD_GETRECORDS_MAX,
+			"Invalid value given for maximum records per getRecords shard operation. Must be a valid non-negative integer value.");
+
+		validateOptionalPositiveIntProperty(config, ConsumerConfigConstants.SHARD_GETRECORDS_RETRIES,
+			"Invalid value given for maximum retry attempts for getRecords shard operation. Must be a valid non-negative integer value.");
+
+		validateOptionalPositiveLongProperty(config, ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_BASE,
+			"Invalid value given for get records operation base backoff milliseconds. Must be a valid non-negative long value.");
+
+		validateOptionalPositiveLongProperty(config, ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_MAX,
+			"Invalid value given for get records operation max backoff milliseconds. Must be a valid non-negative long value.");
+
+		validateOptionalPositiveDoubleProperty(config, ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT,
+			"Invalid value given for get records operation backoff exponential constant. Must be a valid non-negative double value.");
+
+		validateOptionalPositiveLongProperty(config, ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
+			"Invalid value given for getRecords sleep interval in milliseconds. Must be a valid non-negative long value.");
+
+		validateOptionalPositiveIntProperty(config, ConsumerConfigConstants.SHARD_GETITERATOR_RETRIES,
+			"Invalid value given for maximum retry attempts for getShardIterator shard operation. Must be a valid non-negative integer value.");
+
+		validateOptionalPositiveLongProperty(config, ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_BASE,
+			"Invalid value given for get shard iterator operation base backoff milliseconds. Must be a valid non-negative long value.");
+
+		validateOptionalPositiveLongProperty(config, ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_MAX,
+			"Invalid value given for get shard iterator operation max backoff milliseconds. Must be a valid non-negative long value.");
+
+		validateOptionalPositiveDoubleProperty(config, ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT,
+			"Invalid value given for get shard iterator operation backoff exponential constant. Must be a valid non-negative double value.");
+
+		validateOptionalPositiveLongProperty(config, ConsumerConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS,
+			"Invalid value given for shard discovery sleep interval in milliseconds. Must be a valid non-negative long value.");
+
+		validateOptionalPositiveLongProperty(config, ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE,
+			"Invalid value given for describe stream operation base backoff milliseconds. Must be a valid non-negative long value.");
+
+		validateOptionalPositiveLongProperty(config, ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX,
+			"Invalid value given for describe stream operation max backoff milliseconds. Must be a valid non-negative long value.");
+
+		validateOptionalPositiveDoubleProperty(config, ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT,
+			"Invalid value given for describe stream operation backoff exponential constant. Must be a valid non-negative double value.");
+
+		if (config.containsKey(ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS)) {
+			checkArgument(
+				Long.parseLong(config.getProperty(ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS))
+					< ConsumerConfigConstants.MAX_SHARD_GETRECORDS_INTERVAL_MILLIS,
+				"Invalid value given for getRecords sleep interval in milliseconds. Must be lower than " +
+					ConsumerConfigConstants.MAX_SHARD_GETRECORDS_INTERVAL_MILLIS + " milliseconds."
+			);
+		}
+	}
+
+	/**
+	 * Validate configuration properties for {@link FlinkKinesisProducer}.
+	 */
+	public static void validateProducerConfiguration(Properties config) {
+		checkNotNull(config, "config can not be null");
+
+		validateAwsConfiguration(config);
+
+		validateOptionalPositiveLongProperty(config, ProducerConfigConstants.COLLECTION_MAX_COUNT,
+			"Invalid value given for maximum number of items to pack into a PutRecords request. Must be a valid non-negative long value.");
+
+		validateOptionalPositiveLongProperty(config, ProducerConfigConstants.AGGREGATION_MAX_COUNT,
+			"Invalid value given for maximum number of items to pack into an aggregated record. Must be a valid non-negative long value.");
+	}
+
+	/**
+	 * Validate configuration properties related to Amazon AWS service
+	 */
+	public static void validateAwsConfiguration(Properties config) {
+		if (!config.containsKey(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER)) {
+			// if the credential provider type is not specified, it will default to BASIC later on,
+			// so the Access Key ID and Secret Key must be given
+			if (!config.containsKey(AWSConfigConstants.AWS_ACCESS_KEY_ID)
+				|| !config.containsKey(AWSConfigConstants.AWS_SECRET_ACCESS_KEY)) {
+				throw new IllegalArgumentException("Please set values for AWS Access Key ID ('"+ AWSConfigConstants.AWS_ACCESS_KEY_ID +"') " +
+					"and Secret Key ('" + AWSConfigConstants.AWS_SECRET_ACCESS_KEY + "') when using the BASIC AWS credential provider type.");
+			}
+		} else {
+			String credentialsProviderType = config.getProperty(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER);
+
+			// value specified for AWSConfigConstants.AWS_CREDENTIALS_PROVIDER needs to be recognizable
+			CredentialProvider providerType;
+			try {
+				providerType = CredentialProvider.valueOf(credentialsProviderType);
+			} catch (IllegalArgumentException e) {
+				StringBuilder sb = new StringBuilder();
+				for (CredentialProvider type : CredentialProvider.values()) {
+					sb.append(type.toString()).append(", ");
+				}
+				throw new IllegalArgumentException("Invalid AWS Credential Provider Type set in config. Valid values are: " + sb.toString());
+			}
+
+			// if BASIC type is used, also check that the Access Key ID and Secret Key is supplied
+			if (providerType == CredentialProvider.BASIC) {
+				if (!config.containsKey(AWSConfigConstants.AWS_ACCESS_KEY_ID)
+					|| !config.containsKey(AWSConfigConstants.AWS_SECRET_ACCESS_KEY)) {
+					throw new IllegalArgumentException("Please set values for AWS Access Key ID ('"+ AWSConfigConstants.AWS_ACCESS_KEY_ID +"') " +
+						"and Secret Key ('" + AWSConfigConstants.AWS_SECRET_ACCESS_KEY + "') when using the BASIC AWS credential provider type.");
+				}
+			}
+		}
+
+		if (!config.containsKey(AWSConfigConstants.AWS_REGION)) {
+			throw new IllegalArgumentException("The AWS region ('" + AWSConfigConstants.AWS_REGION + "') must be set in the config.");
+		} else {
+			// specified AWS Region name must be recognizable
+			if (!AWSUtil.isValidRegion(config.getProperty(AWSConfigConstants.AWS_REGION))) {
+				StringBuilder sb = new StringBuilder();
+				for (Regions region : Regions.values()) {
+					sb.append(region.getName()).append(", ");
+				}
+				throw new IllegalArgumentException("Invalid AWS region set in config. Valid values are: " + sb.toString());
+			}
+		}
+	}
+
+	private static void validateOptionalPositiveLongProperty(Properties config, String key, String message) {
+		if (config.containsKey(key)) {
+			try {
+				long value = Long.parseLong(config.getProperty(key));
+				if (value < 0) {
+					throw new NumberFormatException();
+				}
+			} catch (NumberFormatException e) {
+				throw new IllegalArgumentException(message);
+			}
+		}
+	}
+
+	private static void validateOptionalPositiveIntProperty(Properties config, String key, String message) {
+		if (config.containsKey(key)) {
+			try {
+				int value = Integer.parseInt(config.getProperty(key));
+				if (value < 0) {
+					throw new NumberFormatException();
+				}
+			} catch (NumberFormatException e) {
+				throw new IllegalArgumentException(message);
+			}
+		}
+	}
+
+	private static void validateOptionalPositiveDoubleProperty(Properties config, String key, String message) {
+		if (config.containsKey(key)) {
+			try {
+				double value = Double.parseDouble(config.getProperty(key));
+				if (value < 0) {
+					throw new NumberFormatException();
+				}
+			} catch (NumberFormatException e) {
+				throw new IllegalArgumentException(message);
+			}
+		}
+	}
+}

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


[22/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
new file mode 100644
index 0000000..3a95d94
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import com.esotericsoftware.kryo.Kryo;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.types.NormalizableKey;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.hadoop.io.Writable;
+import org.objenesis.strategy.StdInstantiatorStrategy;
+
+import java.io.IOException;
+
+public class WritableComparator<T extends Writable & Comparable<T>> extends TypeComparator<T> {
+	
+	private static final long serialVersionUID = 1L;
+	
+	private Class<T> type;
+	
+	private final boolean ascendingComparison;
+	
+	private transient T reference;
+	
+	private transient T tempReference;
+	
+	private transient Kryo kryo;
+
+	@SuppressWarnings("rawtypes")
+	private final TypeComparator[] comparators = new TypeComparator[] {this};
+
+	public WritableComparator(boolean ascending, Class<T> type) {
+		this.type = type;
+		this.ascendingComparison = ascending;
+	}
+	
+	@Override
+	public int hash(T record) {
+		return record.hashCode();
+	}
+	
+	@Override
+	public void setReference(T toCompare) {
+		checkKryoInitialized();
+
+		reference = KryoUtils.copy(toCompare, kryo, new WritableSerializer<T>(type));
+	}
+	
+	@Override
+	public boolean equalToReference(T candidate) {
+		return candidate.equals(reference);
+	}
+	
+	@Override
+	public int compareToReference(TypeComparator<T> referencedComparator) {
+		T otherRef = ((WritableComparator<T>) referencedComparator).reference;
+		int comp = otherRef.compareTo(reference);
+		return ascendingComparison ? comp : -comp;
+	}
+	
+	@Override
+	public int compare(T first, T second) {
+		int comp = first.compareTo(second);
+		return ascendingComparison ? comp : -comp;
+	}
+	
+	@Override
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+		ensureReferenceInstantiated();
+		ensureTempReferenceInstantiated();
+		
+		reference.readFields(firstSource);
+		tempReference.readFields(secondSource);
+		
+		int comp = reference.compareTo(tempReference);
+		return ascendingComparison ? comp : -comp;
+	}
+	
+	@Override
+	public boolean supportsNormalizedKey() {
+		return NormalizableKey.class.isAssignableFrom(type);
+	}
+	
+	@Override
+	public int getNormalizeKeyLen() {
+		ensureReferenceInstantiated();
+		
+		NormalizableKey<?> key = (NormalizableKey<?>) reference;
+		return key.getMaxNormalizedKeyLen();
+	}
+	
+	@Override
+	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+		return keyBytes < getNormalizeKeyLen();
+	}
+	
+	@Override
+	public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
+		NormalizableKey<?> key = (NormalizableKey<?>) record;
+		key.copyNormalizedKey(target, offset, numBytes);
+	}
+	
+	@Override
+	public boolean invertNormalizedKey() {
+		return !ascendingComparison;
+	}
+	
+	@Override
+	public TypeComparator<T> duplicate() {
+		return new WritableComparator<T>(ascendingComparison, type);
+	}
+
+	@Override
+	public int extractKeys(Object record, Object[] target, int index) {
+		target[index] = record;
+		return 1;
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public TypeComparator[] getFlatComparators() {
+		return comparators;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	// unsupported normalization
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public boolean supportsSerializationWithKeyNormalization() {
+		return false;
+	}
+	
+	@Override
+	public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+	
+	@Override
+	public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private void checkKryoInitialized() {
+		if (this.kryo == null) {
+			this.kryo = new Kryo();
+
+			Kryo.DefaultInstantiatorStrategy instantiatorStrategy = new Kryo.DefaultInstantiatorStrategy();
+			instantiatorStrategy.setFallbackInstantiatorStrategy(new StdInstantiatorStrategy());
+			kryo.setInstantiatorStrategy(instantiatorStrategy);
+
+			this.kryo.setAsmEnabled(true);
+			this.kryo.register(type);
+		}
+	}
+	
+	private void ensureReferenceInstantiated() {
+		if (reference == null) {
+			reference = InstantiationUtil.instantiate(type, Writable.class);
+		}
+	}
+	
+	private void ensureTempReferenceInstantiated() {
+		if (tempReference == null) {
+			tempReference = InstantiationUtil.instantiate(type, Writable.class);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
new file mode 100644
index 0000000..9036d75
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils.runtime;
+
+
+import com.esotericsoftware.kryo.Kryo;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.objenesis.strategy.StdInstantiatorStrategy;
+
+import java.io.IOException;
+
+public class WritableSerializer<T extends Writable> extends TypeSerializer<T> {
+	
+	private static final long serialVersionUID = 1L;
+	
+	private final Class<T> typeClass;
+	
+	private transient Kryo kryo;
+	
+	private transient T copyInstance;
+	
+	public WritableSerializer(Class<T> typeClass) {
+		this.typeClass = typeClass;
+	}
+	
+	@SuppressWarnings("unchecked")
+	@Override
+	public T createInstance() {
+		if(typeClass == NullWritable.class) {
+			return (T) NullWritable.get();
+		}
+		return InstantiationUtil.instantiate(typeClass);
+	}
+
+
+	
+	@Override
+	public T copy(T from) {
+		checkKryoInitialized();
+
+		return KryoUtils.copy(from, kryo, this);
+	}
+	
+	@Override
+	public T copy(T from, T reuse) {
+		checkKryoInitialized();
+
+		return KryoUtils.copy(from, reuse, kryo, this);
+	}
+	
+	@Override
+	public int getLength() {
+		return -1;
+	}
+	
+	@Override
+	public void serialize(T record, DataOutputView target) throws IOException {
+		record.write(target);
+	}
+	
+	@Override
+	public T deserialize(DataInputView source) throws IOException {
+		return deserialize(createInstance(), source);
+	}
+	
+	@Override
+	public T deserialize(T reuse, DataInputView source) throws IOException {
+		reuse.readFields(source);
+		return reuse;
+	}
+	
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		ensureInstanceInstantiated();
+		copyInstance.readFields(source);
+		copyInstance.write(target);
+	}
+	
+	@Override
+	public boolean isImmutableType() {
+		return false;
+	}
+	
+	@Override
+	public WritableSerializer<T> duplicate() {
+		return new WritableSerializer<T>(typeClass);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private void ensureInstanceInstantiated() {
+		if (copyInstance == null) {
+			copyInstance = createInstance();
+		}
+	}
+	
+	private void checkKryoInitialized() {
+		if (this.kryo == null) {
+			this.kryo = new Kryo();
+
+			Kryo.DefaultInstantiatorStrategy instantiatorStrategy = new Kryo.DefaultInstantiatorStrategy();
+			instantiatorStrategy.setFallbackInstantiatorStrategy(new StdInstantiatorStrategy());
+			kryo.setInstantiatorStrategy(instantiatorStrategy);
+
+			this.kryo.setAsmEnabled(true);
+			this.kryo.register(typeClass);
+		}
+	}
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public int hashCode() {
+		return this.typeClass.hashCode();
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof WritableSerializer) {
+			WritableSerializer<?> other = (WritableSerializer<?>) obj;
+
+			return other.canEqual(this) && typeClass == other.typeClass;
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof WritableSerializer;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopInputs.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopInputs.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopInputs.java
new file mode 100644
index 0000000..9e8a3e4
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopInputs.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hadoopcompatibility;
+
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Job;
+
+import java.io.IOException;
+
+/**
+ * HadoopInputs is a utility class to use Apache Hadoop InputFormats with Apache Flink.
+ *
+ * It provides methods to create Flink InputFormat wrappers for Hadoop {@link org.apache.hadoop.mapred.InputFormat}
+ * and {@link org.apache.hadoop.mapreduce.InputFormat}.
+ *
+ * Key value pairs produced by the Hadoop InputFormats are converted into Flink
+ * {@link org.apache.flink.api.java.tuple.Tuple2 Tuple2} objects where the first field
+ * ({@link org.apache.flink.api.java.tuple.Tuple2#f0 Tuple2.f0}) is the key and the second field
+ * ({@link org.apache.flink.api.java.tuple.Tuple2#f1 Tuple2.f1}) is the value.
+ *
+ */
+
+public final class HadoopInputs {
+	// ----------------------------------- Hadoop Input Format ---------------------------------------
+
+	/**
+	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapred.FileInputFormat}.
+	 *
+	 * @return A Flink InputFormat that wraps the Hadoop FileInputFormat.
+	 */
+	public static <K,V> HadoopInputFormat<K, V> readHadoopFile(org.apache.hadoop.mapred.FileInputFormat<K,V> mapredInputFormat, Class<K> key, Class<V> value, String inputPath, JobConf job) {
+		// set input path in JobConf
+		org.apache.hadoop.mapred.FileInputFormat.addInputPath(job, new org.apache.hadoop.fs.Path(inputPath));
+		// return wrapping InputFormat
+		return createHadoopInput(mapredInputFormat, key, value, job);
+	}
+
+	/**
+	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapred.FileInputFormat}.
+	 *
+	 * @return A Flink InputFormat that wraps the Hadoop FileInputFormat.
+	 */
+	public static <K,V> HadoopInputFormat<K, V> readHadoopFile(org.apache.hadoop.mapred.FileInputFormat<K,V> mapredInputFormat, Class<K> key, Class<V> value, String inputPath) {
+		return readHadoopFile(mapredInputFormat, key, value, inputPath, new JobConf());
+	}
+
+	/**
+	 * Creates a Flink {@link InputFormat} to read a Hadoop sequence file for the given key and value classes.
+	 *
+	 * @return A Flink InputFormat that wraps a Hadoop SequenceFileInputFormat.
+	 */
+	public static <K,V> HadoopInputFormat<K, V> readSequenceFile(Class<K> key, Class<V> value, String inputPath) throws IOException {
+		return readHadoopFile(new org.apache.hadoop.mapred.SequenceFileInputFormat<K, V>(), key, value, inputPath);
+	}
+
+	/**
+	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapred.InputFormat}.
+	 *
+	 * @return A Flink InputFormat that wraps the Hadoop InputFormat.
+	 */
+	public static <K,V> HadoopInputFormat<K, V> createHadoopInput(org.apache.hadoop.mapred.InputFormat<K,V> mapredInputFormat, Class<K> key, Class<V> value, JobConf job) {
+		return new HadoopInputFormat<>(mapredInputFormat, key, value, job);
+	}
+
+	/**
+	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}.
+	 *
+	 * @return A Flink InputFormat that wraps the Hadoop FileInputFormat.
+	 */
+	public static <K,V> org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat<K, V> readHadoopFile(
+			org.apache.hadoop.mapreduce.lib.input.FileInputFormat<K,V> mapreduceInputFormat, Class<K> key, Class<V> value, String inputPath, Job job) throws IOException
+	{
+		// set input path in Job
+		org.apache.hadoop.mapreduce.lib.input.FileInputFormat.addInputPath(job, new org.apache.hadoop.fs.Path(inputPath));
+		// return wrapping InputFormat
+		return createHadoopInput(mapreduceInputFormat, key, value, job);
+	}
+
+	/**
+	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}.
+	 *
+	 * @return A Flink InputFormat that wraps the Hadoop FileInputFormat.
+	 */
+	public static <K,V> org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat<K, V> readHadoopFile(
+			org.apache.hadoop.mapreduce.lib.input.FileInputFormat<K,V> mapreduceInputFormat, Class<K> key, Class<V> value, String inputPath) throws IOException
+	{
+		return readHadoopFile(mapreduceInputFormat, key, value, inputPath, Job.getInstance());
+	}
+
+	/**
+	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapreduce.InputFormat}.
+	 *
+	 * @return A Flink InputFormat that wraps the Hadoop InputFormat.
+	 */
+	public static <K,V> org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat<K, V> createHadoopInput(
+			org.apache.hadoop.mapreduce.InputFormat<K,V> mapreduceInputFormat, Class<K> key, Class<V> value, Job job)
+	{
+		return new org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat<>(mapreduceInputFormat, key, value, job);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java
new file mode 100644
index 0000000..97ca329
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hadoopcompatibility;
+
+import org.apache.commons.cli.Option;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.hadoop.util.GenericOptionsParser;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Utility class to work with Apache Hadoop libraries.
+ */
+public class HadoopUtils {
+	/**
+	 * Returns {@link ParameterTool} for the arguments parsed by {@link GenericOptionsParser}
+	 *
+	 * @param args Input array arguments. It should be parsable by {@link GenericOptionsParser}
+	 * @return A {@link ParameterTool}
+	 * @throws IOException If arguments cannot be parsed by {@link GenericOptionsParser}
+	 * @see GenericOptionsParser
+	 */
+	public static ParameterTool paramsFromGenericOptionsParser(String[] args) throws IOException {
+		Option[] options = new GenericOptionsParser(args).getCommandLine().getOptions();
+		Map<String, String> map = new HashMap<String, String>();
+		for (Option option : options) {
+			String[] split = option.getValue().split("=");
+			map.put(split[0], split[1]);
+		}
+		return ParameterTool.fromMap(map);
+	}
+}
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
new file mode 100644
index 0000000..ba8aa90
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hadoopcompatibility.mapred;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+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.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
+import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * This wrapper maps a Hadoop Mapper (mapred API) to a Flink FlatMapFunction. 
+ */
+@SuppressWarnings("rawtypes")
+@Public
+public final class HadoopMapFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT> 
+					extends RichFlatMapFunction<Tuple2<KEYIN,VALUEIN>, Tuple2<KEYOUT,VALUEOUT>> 
+					implements ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private transient Mapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT> mapper;
+	private transient JobConf jobConf;
+
+	private transient HadoopOutputCollector<KEYOUT,VALUEOUT> outputCollector;
+	private transient Reporter reporter;
+	
+	/**
+	 * Maps a Hadoop Mapper (mapred API) to a Flink FlatMapFunction.
+	 * 
+	 * @param hadoopMapper The Hadoop Mapper to wrap.
+	 */
+	public HadoopMapFunction(Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopMapper) {
+		this(hadoopMapper, new JobConf());
+	}
+	
+	/**
+	 * Maps a Hadoop Mapper (mapred API) to a Flink FlatMapFunction.
+	 * The Hadoop Mapper is configured with the provided JobConf.
+	 * 
+	 * @param hadoopMapper The Hadoop Mapper to wrap.
+	 * @param conf The JobConf that is used to configure the Hadoop Mapper.
+	 */
+	public HadoopMapFunction(Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopMapper, JobConf conf) {
+		if(hadoopMapper == null) {
+			throw new NullPointerException("Mapper may not be null.");
+		}
+		if(conf == null) {
+			throw new NullPointerException("JobConf may not be null.");
+		}
+		
+		this.mapper = hadoopMapper;
+		this.jobConf = conf;
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		this.mapper.configure(jobConf);
+		
+		this.reporter = new HadoopDummyReporter();
+		this.outputCollector = new HadoopOutputCollector<KEYOUT, VALUEOUT>();
+	}
+
+	@Override
+	public void flatMap(final Tuple2<KEYIN,VALUEIN> value, final Collector<Tuple2<KEYOUT,VALUEOUT>> out) 
+			throws Exception {
+		outputCollector.setFlinkCollector(out);
+		mapper.map(value.f0, value.f1, outputCollector, reporter);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public TypeInformation<Tuple2<KEYOUT,VALUEOUT>> getProducedType() {	
+		Class<KEYOUT> outKeyClass = (Class<KEYOUT>) TypeExtractor.getParameterType(Mapper.class, mapper.getClass(), 2);
+		Class<VALUEOUT> outValClass = (Class<VALUEOUT>)TypeExtractor.getParameterType(Mapper.class, mapper.getClass(), 3);
+		
+		final TypeInformation<KEYOUT> keyTypeInfo = TypeExtractor.getForClass((Class<KEYOUT>) outKeyClass);
+		final TypeInformation<VALUEOUT> valueTypleInfo = TypeExtractor.getForClass((Class<VALUEOUT>) outValClass);
+		return new TupleTypeInfo<Tuple2<KEYOUT,VALUEOUT>>(keyTypeInfo, valueTypleInfo);
+	}
+	
+	/**
+	 * Custom serialization methods.
+	 * @see <a href="http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html">http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html</a>
+	 */
+	private void writeObject(final ObjectOutputStream out) throws IOException {
+		out.writeObject(mapper.getClass());
+		jobConf.write(out);
+	}
+
+	@SuppressWarnings("unchecked")
+	private void readObject(final ObjectInputStream in) throws IOException, ClassNotFoundException {
+		Class<Mapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT>> mapperClass = 
+				(Class<Mapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT>>)in.readObject();
+		mapper = InstantiationUtil.instantiate(mapperClass);
+		
+		jobConf = new JobConf();
+		jobConf.readFields(in);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
new file mode 100644
index 0000000..c1acc2b
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hadoopcompatibility.mapred;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.functions.GroupCombineFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
+import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector;
+import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * This wrapper maps a Hadoop Reducer and Combiner (mapred API) to a combinable Flink GroupReduceFunction.
+ */
+@SuppressWarnings("rawtypes")
+@Public
+public final class HadoopReduceCombineFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT> 
+	extends RichGroupReduceFunction<Tuple2<KEYIN,VALUEIN>,Tuple2<KEYOUT,VALUEOUT>>
+	implements GroupCombineFunction<Tuple2<KEYIN,VALUEIN>, Tuple2<KEYIN,VALUEIN>>,
+				ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private transient Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> reducer;
+	private transient Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN> combiner;
+	private transient JobConf jobConf;
+	
+	private transient HadoopTupleUnwrappingIterator<KEYIN, VALUEIN> valueIterator;
+	private transient HadoopOutputCollector<KEYOUT,VALUEOUT> reduceCollector;
+	private transient HadoopOutputCollector<KEYIN,VALUEIN> combineCollector;
+	private transient Reporter reporter;
+
+	/**
+	 * Maps two Hadoop Reducer (mapred API) to a combinable Flink GroupReduceFunction.
+	 * 
+	 * @param hadoopReducer The Hadoop Reducer that is mapped to a GroupReduceFunction.
+	 * @param hadoopCombiner The Hadoop Reducer that is mapped to the combiner function.
+	 */
+	public HadoopReduceCombineFunction(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopReducer,
+										Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN> hadoopCombiner) {
+		this(hadoopReducer, hadoopCombiner, new JobConf());
+	}
+	
+	/**
+	 * Maps two Hadoop Reducer (mapred API) to a combinable Flink GroupReduceFunction.
+	 * 
+	 * @param hadoopReducer The Hadoop Reducer that is mapped to a GroupReduceFunction.
+	 * @param hadoopCombiner The Hadoop Reducer that is mapped to the combiner function.
+	 * @param conf The JobConf that is used to configure both Hadoop Reducers.
+	 */
+	public HadoopReduceCombineFunction(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopReducer,
+								Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN> hadoopCombiner, JobConf conf) {
+		if(hadoopReducer == null) {
+			throw new NullPointerException("Reducer may not be null.");
+		}
+		if(hadoopCombiner == null) {
+			throw new NullPointerException("Combiner may not be null.");
+		}
+		if(conf == null) {
+			throw new NullPointerException("JobConf may not be null.");
+		}
+		
+		this.reducer = hadoopReducer;
+		this.combiner = hadoopCombiner;
+		this.jobConf = conf;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		this.reducer.configure(jobConf);
+		this.combiner.configure(jobConf);
+		
+		this.reporter = new HadoopDummyReporter();
+		Class<KEYIN> inKeyClass = (Class<KEYIN>) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 0);
+		TypeSerializer<KEYIN> keySerializer = TypeExtractor.getForClass(inKeyClass).createSerializer(getRuntimeContext().getExecutionConfig());
+		this.valueIterator = new HadoopTupleUnwrappingIterator<>(keySerializer);
+		this.combineCollector = new HadoopOutputCollector<>();
+		this.reduceCollector = new HadoopOutputCollector<>();
+	}
+
+	@Override
+	public void reduce(final Iterable<Tuple2<KEYIN,VALUEIN>> values, final Collector<Tuple2<KEYOUT,VALUEOUT>> out)
+			throws Exception {
+		reduceCollector.setFlinkCollector(out);
+		valueIterator.set(values.iterator());
+		reducer.reduce(valueIterator.getCurrentKey(), valueIterator, reduceCollector, reporter);
+	}
+
+	@Override
+	public void combine(final Iterable<Tuple2<KEYIN,VALUEIN>> values, final Collector<Tuple2<KEYIN,VALUEIN>> out) throws Exception {
+		combineCollector.setFlinkCollector(out);
+		valueIterator.set(values.iterator());
+		combiner.reduce(valueIterator.getCurrentKey(), valueIterator, combineCollector, reporter);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public TypeInformation<Tuple2<KEYOUT,VALUEOUT>> getProducedType() {
+		Class<KEYOUT> outKeyClass = (Class<KEYOUT>) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 2);
+		Class<VALUEOUT> outValClass = (Class<VALUEOUT>)TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 3);
+
+		final TypeInformation<KEYOUT> keyTypeInfo = TypeExtractor.getForClass(outKeyClass);
+		final TypeInformation<VALUEOUT> valueTypleInfo = TypeExtractor.getForClass(outValClass);
+		return new TupleTypeInfo<>(keyTypeInfo, valueTypleInfo);
+	}
+
+	/**
+	 * Custom serialization methods.
+	 * @see <a href="http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html">http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html</a>
+	 */
+	private void writeObject(final ObjectOutputStream out) throws IOException {
+		
+		out.writeObject(reducer.getClass());
+		out.writeObject(combiner.getClass());
+		jobConf.write(out);
+	}
+
+	@SuppressWarnings("unchecked")
+	private void readObject(final ObjectInputStream in) throws IOException, ClassNotFoundException {
+		
+		Class<Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT>> reducerClass = 
+				(Class<Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT>>)in.readObject();
+		reducer = InstantiationUtil.instantiate(reducerClass);
+		
+		Class<Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN>> combinerClass = 
+				(Class<Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN>>)in.readObject();
+		combiner = InstantiationUtil.instantiate(combinerClass);
+		
+		jobConf = new JobConf();
+		jobConf.readFields(in);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
new file mode 100644
index 0000000..55aea24
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hadoopcompatibility.mapred;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
+import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector;
+import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * This wrapper maps a Hadoop Reducer (mapred API) to a non-combinable Flink GroupReduceFunction. 
+ */
+@SuppressWarnings("rawtypes")
+@Public
+public final class HadoopReduceFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT> 
+					extends RichGroupReduceFunction<Tuple2<KEYIN,VALUEIN>,Tuple2<KEYOUT,VALUEOUT>> 
+					implements ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private transient Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> reducer;
+	private transient JobConf jobConf;
+	
+	private transient HadoopTupleUnwrappingIterator<KEYIN, VALUEIN> valueIterator;
+	private transient HadoopOutputCollector<KEYOUT,VALUEOUT> reduceCollector;
+	private transient Reporter reporter;
+	
+	/**
+	 * Maps a Hadoop Reducer (mapred API) to a non-combinable Flink GroupReduceFunction.
+ 	 * 
+	 * @param hadoopReducer The Hadoop Reducer to wrap.
+	 */
+	public HadoopReduceFunction(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopReducer) {
+		this(hadoopReducer, new JobConf());
+	}
+	
+	/**
+	 * Maps a Hadoop Reducer (mapred API) to a non-combinable Flink GroupReduceFunction.
+ 	 * 
+	 * @param hadoopReducer The Hadoop Reducer to wrap.
+	 * @param conf The JobConf that is used to configure the Hadoop Reducer.
+	 */
+	public HadoopReduceFunction(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopReducer, JobConf conf) {
+		if(hadoopReducer == null) {
+			throw new NullPointerException("Reducer may not be null.");
+		}
+		if(conf == null) {
+			throw new NullPointerException("JobConf may not be null.");
+		}
+		
+		this.reducer = hadoopReducer;
+		this.jobConf = conf;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		this.reducer.configure(jobConf);
+		
+		this.reporter = new HadoopDummyReporter();
+		this.reduceCollector = new HadoopOutputCollector<KEYOUT, VALUEOUT>();
+		Class<KEYIN> inKeyClass = (Class<KEYIN>) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 0);
+		TypeSerializer<KEYIN> keySerializer = TypeExtractor.getForClass(inKeyClass).createSerializer(getRuntimeContext().getExecutionConfig());
+		this.valueIterator = new HadoopTupleUnwrappingIterator<KEYIN, VALUEIN>(keySerializer);
+	}
+
+	@Override
+	public void reduce(final Iterable<Tuple2<KEYIN,VALUEIN>> values, final Collector<Tuple2<KEYOUT,VALUEOUT>> out)
+			throws Exception {
+		
+		reduceCollector.setFlinkCollector(out);
+		valueIterator.set(values.iterator());
+		reducer.reduce(valueIterator.getCurrentKey(), valueIterator, reduceCollector, reporter);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public TypeInformation<Tuple2<KEYOUT,VALUEOUT>> getProducedType() {
+		Class<KEYOUT> outKeyClass = (Class<KEYOUT>) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 2);
+		Class<VALUEOUT> outValClass = (Class<VALUEOUT>)TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 3);
+
+		final TypeInformation<KEYOUT> keyTypeInfo = TypeExtractor.getForClass((Class<KEYOUT>) outKeyClass);
+		final TypeInformation<VALUEOUT> valueTypleInfo = TypeExtractor.getForClass((Class<VALUEOUT>) outValClass);
+		return new TupleTypeInfo<Tuple2<KEYOUT,VALUEOUT>>(keyTypeInfo, valueTypleInfo);
+	}
+
+	/**
+	 * Custom serialization methods
+	 * @see <a href="http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html">http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html</a>
+	 */
+	private void writeObject(final ObjectOutputStream out) throws IOException {
+		
+		out.writeObject(reducer.getClass());
+		jobConf.write(out);		
+	}
+
+	@SuppressWarnings("unchecked")
+	private void readObject(final ObjectInputStream in) throws IOException, ClassNotFoundException {
+		
+		Class<Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT>> reducerClass = 
+				(Class<Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT>>)in.readObject();
+		reducer = InstantiationUtil.instantiate(reducerClass);
+		
+		jobConf = new JobConf();
+		jobConf.readFields(in);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopOutputCollector.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopOutputCollector.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopOutputCollector.java
new file mode 100644
index 0000000..bfe03d3
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopOutputCollector.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hadoopcompatibility.mapred.wrapper;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+import org.apache.hadoop.mapred.OutputCollector;
+
+import java.io.IOException;
+
+/**
+ * A Hadoop OutputCollector that wraps a Flink OutputCollector.
+ * On each call of collect() the data is forwarded to the wrapped Flink collector.
+ */
+public final class HadoopOutputCollector<KEY,VALUE> implements OutputCollector<KEY,VALUE> {
+
+	private Collector<Tuple2<KEY,VALUE>> flinkCollector;
+
+	private final Tuple2<KEY,VALUE> outTuple = new Tuple2<KEY, VALUE>();
+
+	/**
+	 * Set the wrapped Flink collector.
+	 * 
+	 * @param flinkCollector The wrapped Flink OutputCollector.
+	 */
+	public void setFlinkCollector(Collector<Tuple2<KEY, VALUE>> flinkCollector) {
+		this.flinkCollector = flinkCollector;
+	}
+	
+	/**
+	 * Use the wrapped Flink collector to collect a key-value pair for Flink. 
+	 * 
+	 * @param key the key to collect
+	 * @param val the value to collect
+	 * @throws IOException unexpected of key or value in key-value pair.
+	 */
+	@Override
+	public void collect(final KEY key, final VALUE val) throws IOException {
+		this.outTuple.f0 = key;
+		this.outTuple.f1 = val;
+		this.flinkCollector.collect(outTuple);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java
new file mode 100644
index 0000000..2d204b8
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hadoopcompatibility.mapred.wrapper;
+
+import java.util.Iterator;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.operators.translation.TupleUnwrappingIterator;
+import org.apache.flink.api.java.tuple.Tuple2;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Wraps a Flink Tuple2 (key-value-pair) iterator into an iterator over the second (value) field.
+ */
+public class HadoopTupleUnwrappingIterator<KEY,VALUE> 
+		extends TupleUnwrappingIterator<VALUE, KEY> implements java.io.Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final TypeSerializer<KEY> keySerializer;
+
+	private transient Iterator<Tuple2<KEY,VALUE>> iterator;
+	
+	private transient KEY curKey;
+	private transient VALUE firstValue;
+	private transient boolean atFirst;
+
+	public HadoopTupleUnwrappingIterator(TypeSerializer<KEY> keySerializer) {
+		this.keySerializer = checkNotNull(keySerializer);
+	}
+	
+	/**
+	 * Set the Flink iterator to wrap.
+	 * 
+	 * @param iterator The Flink iterator to wrap.
+	 */
+	@Override
+	public void set(final Iterator<Tuple2<KEY,VALUE>> iterator) {
+		this.iterator = iterator;
+		if(this.hasNext()) {
+			final Tuple2<KEY, VALUE> tuple = iterator.next();
+			this.curKey = keySerializer.copy(tuple.f0);
+			this.firstValue = tuple.f1;
+			this.atFirst = true;
+		} else {
+			this.atFirst = false;
+		}
+	}
+	
+	@Override
+	public boolean hasNext() {
+		if(this.atFirst) {
+			return true;
+		}
+		return iterator.hasNext();
+	}
+	
+	@Override
+	public VALUE next() {
+		if(this.atFirst) {
+			this.atFirst = false;
+			return firstValue;
+		}
+		
+		final Tuple2<KEY, VALUE> tuple = iterator.next();
+		return tuple.f1;
+	}
+	
+	public KEY getCurrentKey() {
+		return this.curKey;
+	}
+	
+	@Override
+	public void remove() {
+		throw new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/main/scala/org/apache/flink/hadoopcompatibility/scala/HadoopInputs.scala
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/scala/org/apache/flink/hadoopcompatibility/scala/HadoopInputs.scala b/flink-connectors/flink-hadoop-compatibility/src/main/scala/org/apache/flink/hadoopcompatibility/scala/HadoopInputs.scala
new file mode 100644
index 0000000..133a5f4
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/scala/org/apache/flink/hadoopcompatibility/scala/HadoopInputs.scala
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.hadoopcompatibility.scala
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.scala.hadoop.mapreduce
+import org.apache.flink.api.scala.hadoop.mapred
+import org.apache.hadoop.fs.{Path => HadoopPath}
+import org.apache.hadoop.mapred.{JobConf, FileInputFormat => MapredFileInputFormat, InputFormat => MapredInputFormat}
+import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => MapreduceFileInputFormat}
+import org.apache.hadoop.mapreduce.{Job, InputFormat => MapreduceInputFormat}
+
+/**
+  * HadoopInputs is a utility class to use Apache Hadoop InputFormats with Apache Flink.
+  *
+  * It provides methods to create Flink InputFormat wrappers for Hadoop
+  * [[org.apache.hadoop.mapred.InputFormat]] and [[org.apache.hadoop.mapreduce.InputFormat]].
+  *
+  * Key value pairs produced by the Hadoop InputFormats are converted into [[Tuple2]] where
+  * the first field is the key and the second field is the value.
+  *
+  */
+object HadoopInputs {
+
+  /**
+    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
+    * [[org.apache.hadoop.mapred.FileInputFormat]].
+    */
+  def readHadoopFile[K, V](
+      mapredInputFormat: MapredFileInputFormat[K, V],
+      key: Class[K],
+      value: Class[V],
+      inputPath: String,
+      job: JobConf)(implicit tpe: TypeInformation[(K, V)]): mapred.HadoopInputFormat[K, V] = {
+
+    // set input path in JobConf
+    MapredFileInputFormat.addInputPath(job, new HadoopPath(inputPath))
+    // wrap mapredInputFormat
+    createHadoopInput(mapredInputFormat, key, value, job)
+  }
+
+  /**
+    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
+    * [[org.apache.hadoop.mapred.FileInputFormat]].
+    */
+  def readHadoopFile[K, V](
+      mapredInputFormat: MapredFileInputFormat[K, V],
+      key: Class[K],
+      value: Class[V],
+      inputPath: String)(implicit tpe: TypeInformation[(K, V)]): mapred.HadoopInputFormat[K, V] = {
+
+    readHadoopFile(mapredInputFormat, key, value, inputPath, new JobConf)
+  }
+
+  /**
+    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that reads a Hadoop sequence
+    * file with the given key and value classes.
+    */
+  def readSequenceFile[K, V](
+      key: Class[K],
+      value: Class[V],
+      inputPath: String)(implicit tpe: TypeInformation[(K, V)]): mapred.HadoopInputFormat[K, V] = {
+
+    readHadoopFile(
+      new org.apache.hadoop.mapred.SequenceFileInputFormat[K, V],
+      key,
+      value,
+      inputPath
+    )
+  }
+
+  /**
+    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
+    * [[org.apache.hadoop.mapred.InputFormat]].
+    */
+  def createHadoopInput[K, V](
+      mapredInputFormat: MapredInputFormat[K, V],
+      key: Class[K],
+      value: Class[V],
+      job: JobConf)(implicit tpe: TypeInformation[(K, V)]): mapred.HadoopInputFormat[K, V] = {
+
+    new mapred.HadoopInputFormat[K, V](mapredInputFormat, key, value, job)
+  }
+
+  /**
+    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
+    * [[org.apache.hadoop.mapreduce.lib.input.FileInputFormat]].
+    */
+  def readHadoopFile[K, V](
+      mapreduceInputFormat: MapreduceFileInputFormat[K, V],
+      key: Class[K],
+      value: Class[V],
+      inputPath: String,
+      job: Job)(implicit tpe: TypeInformation[(K, V)]): mapreduce.HadoopInputFormat[K, V] = {
+
+    // set input path in Job
+    MapreduceFileInputFormat.addInputPath(job, new HadoopPath(inputPath))
+    // wrap mapreduceInputFormat
+    createHadoopInput(mapreduceInputFormat, key, value, job)
+  }
+
+  /**
+    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
+    * [[org.apache.hadoop.mapreduce.lib.input.FileInputFormat]].
+    */
+  def readHadoopFile[K, V](
+      mapreduceInputFormat: MapreduceFileInputFormat[K, V],
+      key: Class[K],
+      value: Class[V],
+      inputPath: String)(implicit tpe: TypeInformation[(K, V)]): mapreduce.HadoopInputFormat[K, V] =
+  {
+    readHadoopFile(mapreduceInputFormat, key, value, inputPath, Job.getInstance)
+  }
+
+  /**
+    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
+    * [[org.apache.hadoop.mapreduce.InputFormat]].
+    */
+  def createHadoopInput[K, V](
+      mapreduceInputFormat: MapreduceInputFormat[K, V],
+      key: Class[K],
+      value: Class[V],
+      job: Job)(implicit tpe: TypeInformation[(K, V)]): mapreduce.HadoopInputFormat[K, V] = {
+
+    new mapreduce.HadoopInputFormat[K, V](mapreduceInputFormat, key, value, job)
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableExtractionTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableExtractionTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableExtractionTest.java
new file mode 100644
index 0000000..2aefd9f
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableExtractionTest.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils;
+
+import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparator;
+
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+@SuppressWarnings("serial")
+public class WritableExtractionTest {
+
+	@Test
+	public void testDetectWritable() {
+		// writable interface itself must not be writable
+		assertFalse(TypeExtractor.isHadoopWritable(Writable.class));
+
+		// various forms of extension
+		assertTrue(TypeExtractor.isHadoopWritable(DirectWritable.class));
+		assertTrue(TypeExtractor.isHadoopWritable(ViaInterfaceExtension.class));
+		assertTrue(TypeExtractor.isHadoopWritable(ViaAbstractClassExtension.class));
+
+		// some non-writables
+		assertFalse(TypeExtractor.isHadoopWritable(String.class));
+		assertFalse(TypeExtractor.isHadoopWritable(List.class));
+		assertFalse(TypeExtractor.isHadoopWritable(WritableComparator.class));
+	}
+
+	@Test
+	public void testCreateWritableInfo() {
+		TypeInformation<DirectWritable> info1 =
+				TypeExtractor.createHadoopWritableTypeInfo(DirectWritable.class);
+		assertEquals(DirectWritable.class, info1.getTypeClass());
+
+		TypeInformation<ViaInterfaceExtension> info2 =
+				TypeExtractor.createHadoopWritableTypeInfo(ViaInterfaceExtension.class);
+		assertEquals(ViaInterfaceExtension.class, info2.getTypeClass());
+
+		TypeInformation<ViaAbstractClassExtension> info3 = 
+				TypeExtractor.createHadoopWritableTypeInfo(ViaAbstractClassExtension.class);
+		assertEquals(ViaAbstractClassExtension.class, info3.getTypeClass());
+	}
+
+	@Test
+	public void testValidateTypeInfo() {
+		// validate unrelated type info
+		TypeExtractor.validateIfWritable(BasicTypeInfo.STRING_TYPE_INFO, String.class);
+
+		// validate writable type info correctly
+		TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
+				DirectWritable.class), DirectWritable.class);
+		TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
+				ViaInterfaceExtension.class), ViaInterfaceExtension.class);
+		TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
+				ViaAbstractClassExtension.class), ViaAbstractClassExtension.class);
+
+		// incorrect case: not writable at all
+		try {
+			TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
+					DirectWritable.class), String.class);
+			fail("should have failed with an exception");
+		} catch (InvalidTypesException e) {
+			// expected
+		}
+
+		// incorrect case: wrong writable
+		try {
+			TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
+					ViaInterfaceExtension.class), DirectWritable.class);
+			fail("should have failed with an exception");
+		} catch (InvalidTypesException e) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testExtractFromFunction() {
+		RichMapFunction<DirectWritable, DirectWritable> function = new RichMapFunction<DirectWritable, DirectWritable>() {
+			@Override
+			public DirectWritable map(DirectWritable value) throws Exception {
+				return null;
+			}
+		};
+
+		TypeInformation<DirectWritable> outType = 
+				TypeExtractor.getMapReturnTypes(function, new WritableTypeInfo<>(DirectWritable.class));
+
+		assertTrue(outType instanceof WritableTypeInfo);
+		assertEquals(DirectWritable.class, outType.getTypeClass());
+	}
+
+	@Test
+	public void testExtractAsPartOfPojo() {
+		PojoTypeInfo<PojoWithWritable> pojoInfo = 
+				(PojoTypeInfo<PojoWithWritable>) TypeExtractor.getForClass(PojoWithWritable.class);
+
+		boolean foundWritable = false;
+		for (int i = 0; i < pojoInfo.getArity(); i++) {
+			PojoField field = pojoInfo.getPojoFieldAt(i);
+			String name = field.getField().getName();
+			
+			if (name.equals("hadoopCitizen")) {
+				if (foundWritable) {
+					fail("already seen");
+				}
+				foundWritable = true;
+				assertEquals(new WritableTypeInfo<>(DirectWritable.class), field.getTypeInformation());
+				assertEquals(DirectWritable.class, field.getTypeInformation().getTypeClass());
+				
+			}
+		}
+		
+		assertTrue("missed the writable type", foundWritable);
+	}
+
+	@Test
+	public void testInputValidationError() {
+
+		RichMapFunction<Writable, String> function = new RichMapFunction<Writable, String>() {
+			@Override
+			public String map(Writable value) throws Exception {
+				return null;
+			}
+		};
+
+		@SuppressWarnings("unchecked")
+		TypeInformation<Writable> inType = 
+				(TypeInformation<Writable>) (TypeInformation<?>) new WritableTypeInfo<>(DirectWritable.class);
+		
+		try {
+			TypeExtractor.getMapReturnTypes(function, inType);
+			fail("exception expected");
+		}
+		catch (InvalidTypesException e) {
+			// right
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  test type classes
+	// ------------------------------------------------------------------------
+
+	public interface ExtendedWritable extends Writable {}
+
+	public static abstract class AbstractWritable implements Writable {}
+
+	public static class DirectWritable implements Writable {
+
+		@Override
+		public void write(DataOutput dataOutput) throws IOException {}
+
+		@Override
+		public void readFields(DataInput dataInput) throws IOException {}
+	}
+
+	public static class ViaInterfaceExtension implements ExtendedWritable {
+
+		@Override
+		public void write(DataOutput dataOutput) throws IOException {}
+
+		@Override
+		public void readFields(DataInput dataInput) throws IOException {}
+	}
+
+	public static class ViaAbstractClassExtension extends AbstractWritable {
+
+		@Override
+		public void write(DataOutput dataOutput) throws IOException {}
+
+		@Override
+		public void readFields(DataInput dataInput) throws IOException {}
+	}
+
+	public static class PojoWithWritable {
+		public String str;
+		public DirectWritable hadoopCitizen;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableInfoParserTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableInfoParserTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableInfoParserTest.java
new file mode 100644
index 0000000..3d2b652
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableInfoParserTest.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.api.java.typeutils;
+
+import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.hadoop.io.Writable;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class WritableInfoParserTest {
+
+	@Test
+	public void testWritableType() {
+		TypeInformation<?> ti = TypeInfoParser.parse(
+				"Writable<org.apache.flink.api.java.typeutils.WritableInfoParserTest$MyWritable>");
+
+		Assert.assertTrue(ti instanceof WritableTypeInfo<?>);
+		Assert.assertEquals(MyWritable.class, ((WritableTypeInfo<?>) ti).getTypeClass());
+	}
+
+	@Test
+	public void testPojoWithWritableType() {
+		TypeInformation<?> ti = TypeInfoParser.parse(
+				"org.apache.flink.api.java.typeutils.WritableInfoParserTest$MyPojo<"
+				+ "basic=Integer,"
+				+ "tuple=Tuple2<String, Integer>,"
+				+ "hadoopCitizen=Writable<org.apache.flink.api.java.typeutils.WritableInfoParserTest$MyWritable>,"
+				+ "array=String[]"
+				+ ">");
+		Assert.assertTrue(ti instanceof PojoTypeInfo);
+		PojoTypeInfo<?> pti = (PojoTypeInfo<?>) ti;
+		Assert.assertEquals("array", pti.getPojoFieldAt(0).getField().getName());
+		Assert.assertTrue(pti.getPojoFieldAt(0).getTypeInformation() instanceof BasicArrayTypeInfo);
+		Assert.assertEquals("basic", pti.getPojoFieldAt(1).getField().getName());
+		Assert.assertTrue(pti.getPojoFieldAt(1).getTypeInformation() instanceof BasicTypeInfo);
+		Assert.assertEquals("hadoopCitizen", pti.getPojoFieldAt(2).getField().getName());
+		Assert.assertTrue(pti.getPojoFieldAt(2).getTypeInformation() instanceof WritableTypeInfo);
+		Assert.assertEquals("tuple", pti.getPojoFieldAt(3).getField().getName());
+		Assert.assertTrue(pti.getPojoFieldAt(3).getTypeInformation() instanceof TupleTypeInfo);
+	}
+	// ------------------------------------------------------------------------
+	//  Test types
+	// ------------------------------------------------------------------------
+
+	public static class MyWritable implements Writable {
+
+		@Override
+		public void write(DataOutput out) throws IOException {}
+
+		@Override
+		public void readFields(DataInput in) throws IOException {}
+	}
+
+	public static class MyPojo {
+		public Integer basic;
+		public Tuple2<String, Integer> tuple;
+		public MyWritable hadoopCitizen;
+		public String[] array;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
new file mode 100644
index 0000000..eb9cdf0
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.hadoop.io.Writable;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class WritableTypeInfoTest extends TestLogger {
+	
+	@Test
+	public void testWritableTypeInfoEquality() {
+		WritableTypeInfo<TestClass> tpeInfo1 = new WritableTypeInfo<>(TestClass.class);
+		WritableTypeInfo<TestClass> tpeInfo2 = new WritableTypeInfo<>(TestClass.class);
+
+		assertEquals(tpeInfo1, tpeInfo2);
+		assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
+	}
+
+	@Test
+	public void testWritableTypeInfoInequality() {
+		WritableTypeInfo<TestClass> tpeInfo1 = new WritableTypeInfo<>(TestClass.class);
+		WritableTypeInfo<AlternateClass> tpeInfo2 = new WritableTypeInfo<>(AlternateClass.class);
+
+		assertNotEquals(tpeInfo1, tpeInfo2);
+	}
+
+	// ------------------------------------------------------------------------
+	//  test types
+	// ------------------------------------------------------------------------
+
+	public static class TestClass implements Writable {
+
+		@Override
+		public void write(DataOutput dataOutput) throws IOException {}
+
+		@Override
+		public void readFields(DataInput dataInput) throws IOException {}
+	}
+
+	public static class AlternateClass implements Writable {
+
+		@Override
+		public void write(DataOutput dataOutput) throws IOException {}
+
+		@Override
+		public void readFields(DataInput dataInput) throws IOException {}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java
new file mode 100644
index 0000000..c32f5da
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import org.apache.hadoop.io.Writable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class StringArrayWritable implements Writable, Comparable<StringArrayWritable> {
+	
+	private String[] array = new String[0];
+	
+	public StringArrayWritable() {
+		super();
+	}
+	
+	public StringArrayWritable(String[] array) {
+		this.array = array;
+	}
+	
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeInt(this.array.length);
+		
+		for(String str : this.array) {
+			byte[] b = str.getBytes();
+			out.writeInt(b.length);
+			out.write(b);
+		}
+	}
+	
+	@Override
+	public void readFields(DataInput in) throws IOException {
+		this.array = new String[in.readInt()];
+		
+		for(int i = 0; i < this.array.length; i++) {
+			byte[] b = new byte[in.readInt()];
+			in.readFully(b);
+			this.array[i] = new String(b);
+		}
+	}
+	
+	@Override
+	public int compareTo(StringArrayWritable o) {
+		if(this.array.length != o.array.length) {
+			return this.array.length - o.array.length;
+		}
+		
+		for(int i = 0; i < this.array.length; i++) {
+			int comp = this.array[i].compareTo(o.array[i]);
+			if(comp != 0) {
+				return comp;
+			}
+		}
+		return 0;
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+		if(!(obj instanceof StringArrayWritable)) {
+			return false;
+		}
+		return this.compareTo((StringArrayWritable) obj) == 0;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java
new file mode 100644
index 0000000..96f844c
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import org.apache.flink.api.common.typeutils.ComparatorTestBase;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+public class WritableComparatorTest extends ComparatorTestBase<StringArrayWritable> {
+	
+	StringArrayWritable[] data = new StringArrayWritable[]{
+			new StringArrayWritable(new String[]{}),
+			new StringArrayWritable(new String[]{""}),
+			new StringArrayWritable(new String[]{"a","a"}),
+			new StringArrayWritable(new String[]{"a","b"}),
+			new StringArrayWritable(new String[]{"c","c"}),
+			new StringArrayWritable(new String[]{"d","f"}),
+			new StringArrayWritable(new String[]{"d","m"}),
+			new StringArrayWritable(new String[]{"z","x"}),
+			new StringArrayWritable(new String[]{"a","a", "a"})
+	};
+	
+	@Override
+	protected TypeComparator<StringArrayWritable> createComparator(boolean ascending) {
+		return new WritableComparator<StringArrayWritable>(ascending, StringArrayWritable.class);
+	}
+	
+	@Override
+	protected TypeSerializer<StringArrayWritable> createSerializer() {
+		return new WritableSerializer<StringArrayWritable>(StringArrayWritable.class);
+	}
+	
+	@Override
+	protected StringArrayWritable[] getSortedTestData() {
+		return data;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java
new file mode 100644
index 0000000..94e759d
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import org.apache.flink.api.common.typeutils.ComparatorTestBase;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+import java.util.UUID;
+
+public class WritableComparatorUUIDTest extends ComparatorTestBase<WritableID> {
+	@Override
+	protected TypeComparator<WritableID> createComparator(boolean ascending) {
+		return new WritableComparator<>(ascending, WritableID.class);
+	}
+
+	@Override
+	protected TypeSerializer<WritableID> createSerializer() {
+		return new WritableSerializer<>(WritableID.class);
+	}
+
+	@Override
+	protected WritableID[] getSortedTestData() {
+		return new WritableID[] {
+			new WritableID(new UUID(0, 0)),
+			new WritableID(new UUID(1, 0)),
+			new WritableID(new UUID(1, 1))
+		};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java
new file mode 100644
index 0000000..4274cf6
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils.runtime;
+
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.UUID;
+
+public class WritableID implements WritableComparable<WritableID> {
+	private UUID uuid;
+
+	public WritableID() {
+		this.uuid = UUID.randomUUID();
+	}
+
+	public WritableID(UUID uuid) {
+		this.uuid = uuid;
+	}
+
+	@Override
+	public int compareTo(WritableID o) {
+		return this.uuid.compareTo(o.uuid);
+	}
+
+	@Override
+	public void write(DataOutput dataOutput) throws IOException {
+		dataOutput.writeLong(uuid.getMostSignificantBits());
+		dataOutput.writeLong(uuid.getLeastSignificantBits());
+	}
+
+	@Override
+	public void readFields(DataInput dataInput) throws IOException {
+		this.uuid = new UUID(dataInput.readLong(), dataInput.readLong());
+	}
+
+	@Override
+	public String toString() {
+		return uuid.toString();
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		WritableID id = (WritableID) o;
+
+		return !(uuid != null ? !uuid.equals(id.uuid) : id.uuid != null);
+	}
+
+	@Override
+	public int hashCode() {
+		return uuid != null ? uuid.hashCode() : 0;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java
new file mode 100644
index 0000000..bb5f4d4
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.SerializerTestInstance;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.api.java.typeutils.WritableTypeInfo;
+import org.junit.Test;
+
+public class WritableSerializerTest {
+	
+	@Test
+	public void testStringArrayWritable() {
+		StringArrayWritable[] data = new StringArrayWritable[]{
+				new StringArrayWritable(new String[]{}),
+				new StringArrayWritable(new String[]{""}),
+				new StringArrayWritable(new String[]{"a","a"}),
+				new StringArrayWritable(new String[]{"a","b"}),
+				new StringArrayWritable(new String[]{"c","c"}),
+				new StringArrayWritable(new String[]{"d","f"}),
+				new StringArrayWritable(new String[]{"d","m"}),
+				new StringArrayWritable(new String[]{"z","x"}),
+				new StringArrayWritable(new String[]{"a","a", "a"})
+		};
+		
+		WritableTypeInfo<StringArrayWritable> writableTypeInfo = (WritableTypeInfo<StringArrayWritable>) TypeExtractor.getForObject(data[0]);
+		WritableSerializer<StringArrayWritable> writableSerializer = (WritableSerializer<StringArrayWritable>) writableTypeInfo.createSerializer(new ExecutionConfig());
+		
+		SerializerTestInstance<StringArrayWritable> testInstance = new SerializerTestInstance<StringArrayWritable>(writableSerializer,writableTypeInfo.getTypeClass(), -1, data);
+		
+		testInstance.testAll();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java
new file mode 100644
index 0000000..2af7730
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import org.apache.flink.api.common.typeutils.SerializerTestBase;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+import java.util.UUID;
+
+public class WritableSerializerUUIDTest extends SerializerTestBase<WritableID> {
+	@Override
+	protected TypeSerializer<WritableID> createSerializer() {
+		return new WritableSerializer<>(WritableID.class);
+	}
+
+	@Override
+	protected int getLength() {
+		return -1;
+	}
+
+	@Override
+	protected Class<WritableID> getTypeClass() {
+		return WritableID.class;
+	}
+
+	@Override
+	protected WritableID[] getTestData() {
+		return new WritableID[] {
+			new WritableID(new UUID(0, 0)),
+			new WritableID(new UUID(1, 0)),
+			new WritableID(new UUID(1, 1))
+		};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java
new file mode 100644
index 0000000..6f7673b
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hadoopcompatibility;
+
+import org.apache.flink.api.java.utils.AbstractParameterToolTest;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class HadoopUtilsTest extends AbstractParameterToolTest {
+
+	@Test
+	public void testParamsFromGenericOptionsParser() throws IOException {
+		ParameterTool parameter = HadoopUtils.paramsFromGenericOptionsParser(new String[]{"-D", "input=myInput", "-DexpectedCount=15"});
+		validate(parameter);
+	}
+}


[33/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml
new file mode 100644
index 0000000..58eb043
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/pom.xml
@@ -0,0 +1,212 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-kafka-base_2.10</artifactId>
+	<name>flink-connector-kafka-base</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<kafka.version>0.8.2.2</kafka.version>
+	</properties>
+
+	<dependencies>
+
+		<!-- core dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project,
+			won't depend on flink-table. -->
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.kafka</groupId>
+			<artifactId>kafka_${scala.binary.version}</artifactId>
+			<version>${kafka.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>com.sun.jmx</groupId>
+					<artifactId>jmxri</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.sun.jdmk</groupId>
+					<artifactId>jmxtools</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>log4j</groupId>
+					<artifactId>log4j</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>slf4j-simple</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>net.sf.jopt-simple</groupId>
+					<artifactId>jopt-simple</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.scala-lang</groupId>
+					<artifactId>scala-reflect</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.scala-lang</groupId>
+					<artifactId>scala-compiler</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.yammer.metrics</groupId>
+					<artifactId>metrics-annotation</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.xerial.snappy</groupId>
+					<artifactId>snappy-java</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<!-- test dependencies -->
+		
+		<!-- force using the latest zkclient -->
+		<dependency>
+			<groupId>com.101tec</groupId>
+			<artifactId>zkclient</artifactId>
+			<version>0.7</version>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-test</artifactId>
+			<version>${curator.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-metrics-jmx</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-minikdc</artifactId>
+			<version>${minikdc.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+	</dependencies>
+
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>com.101tec</groupId>
+				<artifactId>zkclient</artifactId>
+				<version>0.7</version>
+			</dependency>
+		</dependencies>
+	</dependencyManagement>
+	
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+			<!--
+            https://issues.apache.org/jira/browse/DIRSHARED-134
+            Required to pull the Mini-KDC transitive dependency
+            -->
+			<plugin>
+				<groupId>org.apache.felix</groupId>
+				<artifactId>maven-bundle-plugin</artifactId>
+				<version>3.0.1</version>
+				<inherited>true</inherited>
+				<extensions>true</extensions>
+			</plugin>
+		</plugins>
+	</build>
+	
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
new file mode 100644
index 0000000..aef7116
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.common.state.ListState;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+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.KafkaTopicPartitionState;
+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.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+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,
+		ResultTypeQueryable<T>,
+		CheckpointedFunction {
+	private static final long serialVersionUID = -6272159445203409112L;
+
+	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;
+
+	/** Boolean configuration key to disable metrics tracking **/
+	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
+
+	// ------------------------------------------------------------------------
+	//  configuration state, set on the client relevant for all subtasks
+	// ------------------------------------------------------------------------
+
+	private final List<String> topics;
+	
+	/** The schema to convert between Kafka's byte messages, and Flink's objects */
+	protected final KeyedDeserializationSchema<T> deserializer;
+
+	/** The set of topic partitions that the source will read */
+	protected List<KafkaTopicPartition> subscribedPartitions;
+	
+	/** 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;
+
+	private transient ListState<Tuple2<KafkaTopicPartition, Long>> offsetsStateForCheckpoint;
+
+	// ------------------------------------------------------------------------
+	//  runtime state (used individually by each parallel subtask) 
+	// ------------------------------------------------------------------------
+	
+	/** Data for pending but uncommitted offsets */
+	private final LinkedMap pendingOffsetsToCommit = 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 */
+	private transient volatile HashMap<KafkaTopicPartition, Long> restoreToOffset;
+	
+	/** Flag indicating whether the consumer is still running **/
+	private volatile boolean running = true;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Base constructor.
+	 *
+	 * @param deserializer
+	 *           The deserializer to turn raw byte messages into Java/Scala objects.
+	 */
+	public FlinkKafkaConsumerBase(List<String> topics, KeyedDeserializationSchema<T> deserializer) {
+		this.topics = checkNotNull(topics);
+		checkArgument(topics.size() > 0, "You have to define at least one topic.");
+		this.deserializer = checkNotNull(deserializer, "valueDeserializer");
+	}
+
+	/**
+	 * 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.
+	 */
+	protected void setSubscribedPartitions(List<KafkaTopicPartition> allSubscribedPartitions) {
+		checkNotNull(allSubscribedPartitions);
+		this.subscribedPartitions = Collections.unmodifiableList(allSubscribedPartitions);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Configuration
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * 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> assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks<T> assigner) {
+		checkNotNull(assigner);
+		
+		if (this.periodicWatermarkAssigner != null) {
+			throw new IllegalStateException("A periodic watermark emitter has already been set.");
+		}
+		try {
+			ClosureCleaner.clean(assigner, true);
+			this.punctuatedWatermarkAssigner = new SerializedValue<>(assigner);
+			return this;
+		} catch (Exception e) {
+			throw new IllegalArgumentException("The given assigner is not serializable", e);
+		}
+	}
+
+	/**
+	 * 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> assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks<T> assigner) {
+		checkNotNull(assigner);
+		
+		if (this.punctuatedWatermarkAssigner != null) {
+			throw new IllegalStateException("A punctuated watermark emitter has already been set.");
+		}
+		try {
+			ClosureCleaner.clean(assigner, true);
+			this.periodicWatermarkAssigner = new SerializedValue<>(assigner);
+			return this;
+		} catch (Exception e) {
+			throw new IllegalArgumentException("The given assigner is not serializable", e);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Work methods
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void run(SourceContext<T> sourceContext) throws Exception {
+		if (subscribedPartitions == null) {
+			throw new Exception("The partitions were not set for the consumer");
+		}
+
+		// we need only do work, if we actually have partitions assigned
+		if (!subscribedPartitions.isEmpty()) {
+
+			// (1) create the fetcher that will communicate with the Kafka brokers
+			final AbstractFetcher<T, ?> fetcher = createFetcher(
+					sourceContext, subscribedPartitions,
+					periodicWatermarkAssigner, punctuatedWatermarkAssigner,
+					(StreamingRuntimeContext) getRuntimeContext());
+
+			// (2) set the fetcher to the restored checkpoint offsets
+			if (restoreToOffset != null) {
+				fetcher.restoreOffsets(restoreToOffset);
+			}
+
+			// 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();
+		}
+		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();
+					}
+				}
+			}
+		}
+	}
+
+	@Override
+	public void cancel() {
+		// set ourselves as not running
+		running = false;
+		
+		// abort the fetcher, if there is one
+		if (kafkaFetcher != null) {
+			kafkaFetcher.cancel();
+		}
+
+		// there will be an interrupt() call to the main thread anyways
+	}
+
+	@Override
+	public void open(Configuration configuration) {
+		List<KafkaTopicPartition> kafkaTopicPartitions = getKafkaPartitions(topics);
+
+		if (kafkaTopicPartitions != null) {
+			assignTopicPartitions(kafkaTopicPartitions);
+		}
+	}
+
+	@Override
+	public void close() throws Exception {
+		// pretty much the same logic as cancelling
+		try {
+			cancel();
+		} finally {
+			super.close();
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Checkpoint and restore
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void initializeState(FunctionInitializationContext context) throws Exception {
+
+		OperatorStateStore stateStore = context.getOperatorStateStore();
+		offsetsStateForCheckpoint = stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME);
+
+		if (context.isRestored()) {
+			restoreToOffset = new HashMap<>();
+			for (Tuple2<KafkaTopicPartition, Long> kafkaOffset : offsetsStateForCheckpoint.get()) {
+				restoreToOffset.put(kafkaOffset.f0, kafkaOffset.f1);
+			}
+
+			LOG.info("Setting restore state in the FlinkKafkaConsumer.");
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Using the following offsets: {}", restoreToOffset);
+			}
+		} else {
+			LOG.info("No restore state for FlinkKafkaConsumer.");
+		}
+	}
+
+	@Override
+	public void snapshotState(FunctionSnapshotContext context) throws Exception {
+		if (!running) {
+			LOG.debug("snapshotState() called on closed source");
+		} else {
+
+			offsetsStateForCheckpoint.clear();
+
+			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 or the assigned partitions
+
+				if (restoreToOffset != null) {
+
+					for (Map.Entry<KafkaTopicPartition, Long> kafkaTopicPartitionLongEntry : restoreToOffset.entrySet()) {
+						offsetsStateForCheckpoint.add(
+								Tuple2.of(kafkaTopicPartitionLongEntry.getKey(), kafkaTopicPartitionLongEntry.getValue()));
+					}
+				} else if (subscribedPartitions != null) {
+					for (KafkaTopicPartition subscribedPartition : subscribedPartitions) {
+						offsetsStateForCheckpoint.add(
+								Tuple2.of(subscribedPartition, KafkaTopicPartitionState.OFFSET_NOT_SET));
+					}
+				}
+
+				// the map cannot be asynchronously updated, because only one checkpoint call can happen
+				// on this function at a time: either snapshotState() or notifyCheckpointComplete()
+				pendingOffsetsToCommit.put(context.getCheckpointId(), restoreToOffset);
+			} else {
+				HashMap<KafkaTopicPartition, Long> currentOffsets = fetcher.snapshotCurrentState();
+
+				// the map cannot be asynchronously updated, because only one checkpoint call can happen
+				// on this function at a time: either snapshotState() or notifyCheckpointComplete()
+				pendingOffsetsToCommit.put(context.getCheckpointId(), currentOffsets);
+
+				for (Map.Entry<KafkaTopicPartition, Long> kafkaTopicPartitionLongEntry : currentOffsets.entrySet()) {
+					offsetsStateForCheckpoint.add(
+							Tuple2.of(kafkaTopicPartitionLongEntry.getKey(), kafkaTopicPartitionLongEntry.getValue()));
+				}
+			}
+
+			// truncate the map of pending offsets to commit, to prevent infinite growth
+			while (pendingOffsetsToCommit.size() > MAX_NUM_PENDING_CHECKPOINTS) {
+				pendingOffsetsToCommit.remove(0);
+			}
+		}
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {
+		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 to Kafka/ZooKeeper for checkpoint " + checkpointId);
+		}
+
+		try {
+			final int posInMap = pendingOffsetsToCommit.indexOf(checkpointId);
+			if (posInMap == -1) {
+				LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
+				return;
+			}
+
+			@SuppressWarnings("unchecked")
+			HashMap<KafkaTopicPartition, Long> offsets =
+					(HashMap<KafkaTopicPartition, Long>) pendingOffsetsToCommit.remove(posInMap);
+
+			// remove older checkpoints in map
+			for (int i = 0; i < posInMap; i++) {
+				pendingOffsetsToCommit.remove(0);
+			}
+
+			if (offsets == null || offsets.size() == 0) {
+				LOG.debug("Checkpoint state was empty.");
+				return;
+			}
+			fetcher.commitInternalOffsetsToKafka(offsets);
+		}
+		catch (Exception e) {
+			if (running) {
+				throw e;
+			}
+			// else ignore exception if we are no longer running
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  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;
+
+	protected abstract List<KafkaTopicPartition> getKafkaPartitions(List<String> topics);
+	
+	// ------------------------------------------------------------------------
+	//  ResultTypeQueryable methods 
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public TypeInformation<T> getProducedType() {
+		return deserializer.getProducedType();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private void assignTopicPartitions(List<KafkaTopicPartition> kafkaTopicPartitions) {
+		subscribedPartitions = new ArrayList<>();
+
+		if (restoreToOffset != null) {
+			for (KafkaTopicPartition kafkaTopicPartition : kafkaTopicPartitions) {
+				if (restoreToOffset.containsKey(kafkaTopicPartition)) {
+					subscribedPartitions.add(kafkaTopicPartition);
+				}
+			}
+		} else {
+			Collections.sort(kafkaTopicPartitions, new Comparator<KafkaTopicPartition>() {
+				@Override
+				public int compare(KafkaTopicPartition o1, KafkaTopicPartition o2) {
+					int topicComparison = o1.getTopic().compareTo(o2.getTopic());
+
+					if (topicComparison == 0) {
+						return o1.getPartition() - o2.getPartition();
+					} else {
+						return topicComparison;
+					}
+				}
+			});
+
+			for (int i = getRuntimeContext().getIndexOfThisSubtask(); i < kafkaTopicPartitions.size(); i += getRuntimeContext().getNumberOfParallelSubtasks()) {
+				subscribedPartitions.add(kafkaTopicPartitions.get(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) {
+				thisSubtaskPartitions.add(allPartitions.get(i));
+			}
+		}
+		
+		return thisSubtaskPartitions;
+	}
+	
+	/**
+	 * Logs the partition information in INFO level.
+	 * 
+	 * @param logger The logger to log to.
+	 * @param partitionInfos List of subscribed partitions
+	 */
+	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());
+			if (count == null) {
+				count = 1;
+			} else {
+				count++;
+			}
+			countPerTopic.put(partition.getTopic(), count);
+		}
+		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("), ");
+		}
+		
+		logger.info(sb.toString());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
new file mode 100644
index 0000000..d413f1c
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
@@ -0,0 +1,386 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.runtime.util.SerializableObject;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.util.NetUtils;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Collections;
+import java.util.Comparator;
+
+import static java.util.Objects.requireNonNull;
+
+
+/**
+ * Flink Sink to produce data into a Kafka topic.
+ *
+ * Please note that this producer provides at-least-once reliability guarantees when
+ * checkpoints are enabled and setFlushOnCheckpoint(true) is set.
+ * Otherwise, the producer doesn't provide any reliability guarantees.
+ *
+ * @param <IN> Type of the messages to write into Kafka.
+ */
+public abstract class FlinkKafkaProducerBase<IN> extends RichSinkFunction<IN> implements CheckpointedFunction {
+
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Configuration key for disabling the metrics reporting
+	 */
+	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
+
+	/**
+	 * Array with the partition ids of the given defaultTopicId
+	 * The size of this array is the number of partitions
+	 */
+	protected int[] partitions;
+
+	/**
+	 * User defined properties for the Producer
+	 */
+	protected final Properties producerConfig;
+
+	/**
+	 * The name of the default topic this producer is writing data to
+	 */
+	protected final String defaultTopicId;
+
+	/**
+	 * (Serializable) SerializationSchema for turning objects used with Flink into
+	 * byte[] for Kafka.
+	 */
+	protected final KeyedSerializationSchema<IN> schema;
+
+	/**
+	 * User-provided partitioner for assigning an object to a Kafka partition.
+	 */
+	protected final KafkaPartitioner<IN> partitioner;
+
+	/**
+	 * Flag indicating whether to accept failures (and log them), or to fail on failures
+	 */
+	protected boolean logFailuresOnly;
+
+	/**
+	 * If true, the producer will wait until all outstanding records have been send to the broker.
+	 */
+	protected boolean flushOnCheckpoint;
+	
+	// -------------------------------- Runtime fields ------------------------------------------
+
+	/** KafkaProducer instance */
+	protected transient KafkaProducer<byte[], byte[]> producer;
+
+	/** The callback than handles error propagation or logging callbacks */
+	protected transient Callback callback;
+
+	/** Errors encountered in the async producer are stored here */
+	protected transient volatile Exception asyncException;
+
+	/** Lock for accessing the pending records */
+	protected final SerializableObject pendingRecordsLock = new SerializableObject();
+
+	/** Number of unacknowledged records. */
+	protected long pendingRecords;
+
+	protected OperatorStateStore stateStore;
+
+
+	/**
+	 * The main constructor for creating a FlinkKafkaProducer.
+	 *
+	 * @param defaultTopicId The default topic to write data to
+	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
+	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner
+	 */
+	public FlinkKafkaProducerBase(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
+		requireNonNull(defaultTopicId, "TopicID not set");
+		requireNonNull(serializationSchema, "serializationSchema not set");
+		requireNonNull(producerConfig, "producerConfig not set");
+		ClosureCleaner.clean(customPartitioner, true);
+		ClosureCleaner.ensureSerializable(serializationSchema);
+
+		this.defaultTopicId = defaultTopicId;
+		this.schema = serializationSchema;
+		this.producerConfig = producerConfig;
+
+		// set the producer configuration properties for kafka record key value serializers.
+		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
+			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
+		} else {
+			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
+		}
+
+		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
+			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
+		} else {
+			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
+		}
+
+		// eagerly ensure that bootstrap servers are set.
+		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
+			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
+		}
+
+		this.partitioner = customPartitioner;
+	}
+
+	// ---------------------------------- Properties --------------------------
+
+	/**
+	 * Defines whether the producer should fail on errors, or only log them.
+	 * If this is set to true, then exceptions will be only logged, if set to false,
+	 * exceptions will be eventually thrown and cause the streaming program to 
+	 * fail (and enter recovery).
+	 * 
+	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
+	 */
+	public void setLogFailuresOnly(boolean logFailuresOnly) {
+		this.logFailuresOnly = logFailuresOnly;
+	}
+
+	/**
+	 * If set to true, the Flink producer will wait for all outstanding messages in the Kafka buffers
+	 * to be acknowledged by the Kafka producer on a checkpoint.
+	 * This way, the producer can guarantee that messages in the Kafka buffers are part of the checkpoint.
+	 *
+	 * @param flush Flag indicating the flushing mode (true = flush on checkpoint)
+	 */
+	public void setFlushOnCheckpoint(boolean flush) {
+		this.flushOnCheckpoint = flush;
+	}
+
+	/**
+	 * Used for testing only
+	 */
+	protected <K,V> KafkaProducer<K,V> getKafkaProducer(Properties props) {
+		return new KafkaProducer<>(props);
+	}
+
+	// ----------------------------------- Utilities --------------------------
+	
+	/**
+	 * Initializes the connection to Kafka.
+	 */
+	@Override
+	public void open(Configuration configuration) {
+		producer = getKafkaProducer(this.producerConfig);
+
+		// the fetched list is immutable, so we're creating a mutable copy in order to sort it
+		List<PartitionInfo> partitionsList = new ArrayList<>(producer.partitionsFor(defaultTopicId));
+
+		// sort the partitions by partition id to make sure the fetched partition list is the same across subtasks
+		Collections.sort(partitionsList, new Comparator<PartitionInfo>() {
+			@Override
+			public int compare(PartitionInfo o1, PartitionInfo o2) {
+				return Integer.compare(o1.partition(), o2.partition());
+			}
+		});
+
+		partitions = new int[partitionsList.size()];
+		for (int i = 0; i < partitions.length; i++) {
+			partitions[i] = partitionsList.get(i).partition();
+		}
+
+		RuntimeContext ctx = getRuntimeContext();
+		if (partitioner != null) {
+			partitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), partitions);
+		}
+
+		LOG.info("Starting FlinkKafkaProducer ({}/{}) to produce into topic {}", 
+				ctx.getIndexOfThisSubtask() + 1, ctx.getNumberOfParallelSubtasks(), defaultTopicId);
+
+		// register Kafka metrics to Flink accumulators
+		if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) {
+			Map<MetricName, ? extends Metric> metrics = this.producer.metrics();
+
+			if (metrics == null) {
+				// MapR's Kafka implementation returns null here.
+				LOG.info("Producer implementation does not support metrics");
+			} else {
+				final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer");
+				for (Map.Entry<MetricName, ? extends Metric> metric: metrics.entrySet()) {
+					kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue()));
+				}
+			}
+		}
+
+		if (flushOnCheckpoint && !((StreamingRuntimeContext)this.getRuntimeContext()).isCheckpointingEnabled()) {
+			LOG.warn("Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing.");
+			flushOnCheckpoint = false;
+		}
+
+		if (logFailuresOnly) {
+			callback = new Callback() {
+				@Override
+				public void onCompletion(RecordMetadata metadata, Exception e) {
+					if (e != null) {
+						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
+					}
+					acknowledgeMessage();
+				}
+			};
+		}
+		else {
+			callback = new Callback() {
+				@Override
+				public void onCompletion(RecordMetadata metadata, Exception exception) {
+					if (exception != null && asyncException == null) {
+						asyncException = exception;
+					}
+					acknowledgeMessage();
+				}
+			};
+		}
+	}
+
+	/**
+	 * Called when new data arrives to the sink, and forwards it to Kafka.
+	 *
+	 * @param next
+	 * 		The incoming data
+	 */
+	@Override
+	public void invoke(IN next) throws Exception {
+		// propagate asynchronous errors
+		checkErroneous();
+
+		byte[] serializedKey = schema.serializeKey(next);
+		byte[] serializedValue = schema.serializeValue(next);
+		String targetTopic = schema.getTargetTopic(next);
+		if (targetTopic == null) {
+			targetTopic = defaultTopicId;
+		}
+
+		ProducerRecord<byte[], byte[]> record;
+		if (partitioner == null) {
+			record = new ProducerRecord<>(targetTopic, serializedKey, serializedValue);
+		} else {
+			record = new ProducerRecord<>(targetTopic, partitioner.partition(next, serializedKey, serializedValue, partitions.length), serializedKey, serializedValue);
+		}
+		if (flushOnCheckpoint) {
+			synchronized (pendingRecordsLock) {
+				pendingRecords++;
+			}
+		}
+		producer.send(record, callback);
+	}
+
+
+	@Override
+	public void close() throws Exception {
+		if (producer != null) {
+			producer.close();
+		}
+		
+		// make sure we propagate pending errors
+		checkErroneous();
+	}
+
+	// ------------------- Logic for handling checkpoint flushing -------------------------- //
+
+	private void acknowledgeMessage() {
+		if (flushOnCheckpoint) {
+			synchronized (pendingRecordsLock) {
+				pendingRecords--;
+				if (pendingRecords == 0) {
+					pendingRecordsLock.notifyAll();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Flush pending records.
+	 */
+	protected abstract void flush();
+
+	@Override
+	public void initializeState(FunctionInitializationContext context) throws Exception {
+		this.stateStore = context.getOperatorStateStore();
+	}
+
+	@Override
+	public void snapshotState(FunctionSnapshotContext ctx) throws Exception {
+		if (flushOnCheckpoint) {
+			// flushing is activated: We need to wait until pendingRecords is 0
+			flush();
+			synchronized (pendingRecordsLock) {
+				if (pendingRecords != 0) {
+					throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecords);
+				}
+				// pending records count is 0. We can now confirm the checkpoint
+			}
+		}
+	}
+
+	// ----------------------------------- Utilities --------------------------
+
+	protected void checkErroneous() throws Exception {
+		Exception e = asyncException;
+		if (e != null) {
+			// prevent double throwing
+			asyncException = null;
+			throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e);
+		}
+	}
+	
+	public static Properties getPropertiesFromBrokerList(String brokerList) {
+		String[] elements = brokerList.split(",");
+		
+		// validate the broker addresses
+		for (String broker: elements) {
+			NetUtils.getCorrectHostnamePort(broker);
+		}
+		
+		Properties props = new Properties();
+		props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList);
+		return props;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
new file mode 100644
index 0000000..ee98783
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Base class for {@link KafkaTableSink} that serializes data in JSON format
+ */
+public abstract class KafkaJsonTableSink extends KafkaTableSink {
+
+	/**
+	 * Creates KafkaJsonTableSink
+	 *
+	 * @param topic topic in Kafka to which table is written
+	 * @param properties properties to connect to Kafka
+	 * @param partitioner Kafka partitioner
+	 */
+	public KafkaJsonTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner) {
+		super(topic, properties, partitioner);
+	}
+
+	@Override
+	protected SerializationSchema<Row> createSerializationSchema(String[] fieldNames) {
+		return new JsonRowSerializationSchema(fieldNames);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
new file mode 100644
index 0000000..f145509
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * A version-agnostic Kafka JSON {@link StreamTableSource}.
+ *
+ * <p>The version-specific Kafka consumers need to extend this class and
+ * override {@link #getKafkaConsumer(String, Properties, DeserializationSchema)}}.
+ *
+ * <p>The field names are used to parse the JSON file and so are the types.
+ */
+public abstract class KafkaJsonTableSource extends KafkaTableSource {
+
+	/**
+	 * Creates a generic Kafka JSON {@link StreamTableSource}.
+	 *
+	 * @param topic      Kafka topic to consume.
+	 * @param properties Properties for the Kafka consumer.
+	 * @param fieldNames Row field names.
+	 * @param fieldTypes Row field types.
+	 */
+	KafkaJsonTableSource(
+			String topic,
+			Properties properties,
+			String[] fieldNames,
+			Class<?>[] fieldTypes) {
+
+		super(topic, properties, createDeserializationSchema(fieldNames, fieldTypes), fieldNames, fieldTypes);
+	}
+
+	/**
+	 * Creates a generic Kafka JSON {@link StreamTableSource}.
+	 *
+	 * @param topic      Kafka topic to consume.
+	 * @param properties Properties for the Kafka consumer.
+	 * @param fieldNames Row field names.
+	 * @param fieldTypes Row field types.
+	 */
+	KafkaJsonTableSource(
+			String topic,
+			Properties properties,
+			String[] fieldNames,
+			TypeInformation<?>[] fieldTypes) {
+
+		super(topic, properties, createDeserializationSchema(fieldNames, fieldTypes), fieldNames, fieldTypes);
+	}
+
+	/**
+	 * Configures the failure behaviour if a JSON field is missing.
+	 *
+	 * <p>By default, a missing field is ignored and the field is set to null.
+	 *
+	 * @param failOnMissingField Flag indicating whether to fail or not on a missing field.
+	 */
+	public void setFailOnMissingField(boolean failOnMissingField) {
+		JsonRowDeserializationSchema deserializationSchema = (JsonRowDeserializationSchema) getDeserializationSchema();
+		deserializationSchema.setFailOnMissingField(failOnMissingField);
+	}
+
+	private static JsonRowDeserializationSchema createDeserializationSchema(
+			String[] fieldNames,
+			TypeInformation<?>[] fieldTypes) {
+
+		return new JsonRowDeserializationSchema(fieldNames, fieldTypes);
+	}
+
+	private static JsonRowDeserializationSchema createDeserializationSchema(
+			String[] fieldNames,
+			Class<?>[] fieldTypes) {
+
+		return new JsonRowDeserializationSchema(fieldNames, fieldTypes);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
new file mode 100644
index 0000000..714d9cd
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.sinks.StreamTableSink;
+import org.apache.flink.api.table.typeutils.RowTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Properties;
+
+/**
+ * A version-agnostic Kafka {@link StreamTableSink}.
+ *
+ * <p>The version-specific Kafka consumers need to extend this class and
+ * override {@link #createKafkaProducer(String, Properties, SerializationSchema, KafkaPartitioner)}}.
+ */
+public abstract class KafkaTableSink implements StreamTableSink<Row> {
+
+	protected final String topic;
+	protected final Properties properties;
+	protected SerializationSchema<Row> serializationSchema;
+	protected final KafkaPartitioner<Row> partitioner;
+	protected String[] fieldNames;
+	protected TypeInformation[] fieldTypes;
+
+	/**
+	 * Creates KafkaTableSink
+	 *
+	 * @param topic                 Kafka topic to write to.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param partitioner           Partitioner to select Kafka partition for each item
+	 */
+	public KafkaTableSink(
+			String topic,
+			Properties properties,
+			KafkaPartitioner<Row> partitioner) {
+
+		this.topic = Preconditions.checkNotNull(topic, "topic");
+		this.properties = Preconditions.checkNotNull(properties, "properties");
+		this.partitioner = Preconditions.checkNotNull(partitioner, "partitioner");
+	}
+
+	/**
+	 * Returns the version-specifid Kafka producer.
+	 *
+	 * @param topic               Kafka topic to produce to.
+	 * @param properties          Properties for the Kafka producer.
+	 * @param serializationSchema Serialization schema to use to create Kafka records.
+	 * @param partitioner         Partitioner to select Kafka partition.
+	 * @return The version-specific Kafka producer
+	 */
+	protected abstract FlinkKafkaProducerBase<Row> createKafkaProducer(
+		String topic, Properties properties,
+		SerializationSchema<Row> serializationSchema,
+		KafkaPartitioner<Row> partitioner);
+
+	/**
+	 * Create serialization schema for converting table rows into bytes.
+	 *
+	 * @param fieldNames Field names in table rows.
+	 * @return Instance of serialization schema
+	 */
+	protected abstract SerializationSchema<Row> createSerializationSchema(String[] fieldNames);
+
+	/**
+	 * Create a deep copy of this sink.
+	 *
+	 * @return Deep copy of this sink
+	 */
+	protected abstract KafkaTableSink createCopy();
+
+	@Override
+	public void emitDataStream(DataStream<Row> dataStream) {
+		FlinkKafkaProducerBase<Row> kafkaProducer = createKafkaProducer(topic, properties, serializationSchema, partitioner);
+		dataStream.addSink(kafkaProducer);
+	}
+
+	@Override
+	public TypeInformation<Row> getOutputType() {
+		return new RowTypeInfo(getFieldTypes());
+	}
+
+	public String[] getFieldNames() {
+		return fieldNames;
+	}
+
+	@Override
+	public TypeInformation<?>[] getFieldTypes() {
+		return fieldTypes;
+	}
+
+	@Override
+	public KafkaTableSink configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
+		KafkaTableSink copy = createCopy();
+		copy.fieldNames = Preconditions.checkNotNull(fieldNames, "fieldNames");
+		copy.fieldTypes = Preconditions.checkNotNull(fieldTypes, "fieldTypes");
+		Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
+			"Number of provided field names and types does not match.");
+		copy.serializationSchema = createSerializationSchema(fieldNames);
+
+		return copy;
+	}
+
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
new file mode 100644
index 0000000..fd423d7
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.api.table.typeutils.RowTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Properties;
+
+import static org.apache.flink.streaming.connectors.kafka.internals.TypeUtil.toTypeInfo;
+
+/**
+ * A version-agnostic Kafka {@link StreamTableSource}.
+ *
+ * <p>The version-specific Kafka consumers need to extend this class and
+ * override {@link #getKafkaConsumer(String, Properties, DeserializationSchema)}}.
+ */
+public abstract class KafkaTableSource implements StreamTableSource<Row> {
+
+	/** The Kafka topic to consume. */
+	private final String topic;
+
+	/** Properties for the Kafka consumer. */
+	private final Properties properties;
+
+	/** Deserialization schema to use for Kafka records. */
+	private final DeserializationSchema<Row> deserializationSchema;
+
+	/** Row field names. */
+	private final String[] fieldNames;
+
+	/** Row field types. */
+	private final TypeInformation<?>[] fieldTypes;
+
+	/**
+	 * Creates a generic Kafka {@link StreamTableSource}.
+	 *
+	 * @param topic                 Kafka topic to consume.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param deserializationSchema Deserialization schema to use for Kafka records.
+	 * @param fieldNames            Row field names.
+	 * @param fieldTypes            Row field types.
+	 */
+	KafkaTableSource(
+			String topic,
+			Properties properties,
+			DeserializationSchema<Row> deserializationSchema,
+			String[] fieldNames,
+			Class<?>[] fieldTypes) {
+
+		this(topic, properties, deserializationSchema, fieldNames, toTypeInfo(fieldTypes));
+	}
+
+	/**
+	 * Creates a generic Kafka {@link StreamTableSource}.
+	 *
+	 * @param topic                 Kafka topic to consume.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param deserializationSchema Deserialization schema to use for Kafka records.
+	 * @param fieldNames            Row field names.
+	 * @param fieldTypes            Row field types.
+	 */
+	KafkaTableSource(
+			String topic,
+			Properties properties,
+			DeserializationSchema<Row> deserializationSchema,
+			String[] fieldNames,
+			TypeInformation<?>[] fieldTypes) {
+
+		this.topic = Preconditions.checkNotNull(topic, "Topic");
+		this.properties = Preconditions.checkNotNull(properties, "Properties");
+		this.deserializationSchema = Preconditions.checkNotNull(deserializationSchema, "Deserialization schema");
+		this.fieldNames = Preconditions.checkNotNull(fieldNames, "Field names");
+		this.fieldTypes = Preconditions.checkNotNull(fieldTypes, "Field types");
+
+		Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
+				"Number of provided field names and types does not match.");
+	}
+
+	/**
+	 * NOTE: This method is for internal use only for defining a TableSource.
+	 *       Do not use it in Table API programs.
+	 */
+	@Override
+	public DataStream<Row> getDataStream(StreamExecutionEnvironment env) {
+		// Version-specific Kafka consumer
+		FlinkKafkaConsumerBase<Row> kafkaConsumer = getKafkaConsumer(topic, properties, deserializationSchema);
+		DataStream<Row> kafkaSource = env.addSource(kafkaConsumer);
+		return kafkaSource;
+	}
+
+	@Override
+	public int getNumberOfFields() {
+		return fieldNames.length;
+	}
+
+	@Override
+	public String[] getFieldsNames() {
+		return fieldNames;
+	}
+
+	@Override
+	public TypeInformation<?>[] getFieldTypes() {
+		return fieldTypes;
+	}
+
+	@Override
+	public TypeInformation<Row> getReturnType() {
+		return new RowTypeInfo(fieldTypes);
+	}
+
+	/**
+	 * Returns the version-specific Kafka consumer.
+	 *
+	 * @param topic                 Kafka topic to consume.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param deserializationSchema Deserialization schema to use for Kafka records.
+	 * @return The version-specific Kafka consumer
+	 */
+	abstract FlinkKafkaConsumerBase<Row> getKafkaConsumer(
+			String topic,
+			Properties properties,
+			DeserializationSchema<Row> deserializationSchema);
+
+	/**
+	 * Returns the deserialization schema.
+	 *
+	 * @return The deserialization schema
+	 */
+	protected DeserializationSchema<Row> getDeserializationSchema() {
+		return deserializationSchema;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
new file mode 100644
index 0000000..cf39606
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+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.watermark.Watermark;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+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> {
+	
+	protected static final int NO_TIMESTAMPS_WATERMARKS = 0;
+	protected static final int PERIODIC_WATERMARKS = 1;
+	protected static final int PUNCTUATED_WATERMARKS = 2;
+	
+	// ------------------------------------------------------------------------
+	
+	/** The source context to emit records and watermarks to */
+	protected final SourceContext<T> sourceContext;
+
+	/** The lock that guarantees that record emission and state updates are atomic,
+	 * from the view of taking a checkpoint */
+	protected 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 */
+	protected final int timestampWatermarkMode;
+
+	/** Flag whether to register metrics for the fetcher */
+	protected final boolean useMetrics;
+
+	/** 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,
+			ProcessingTimeService processingTimeProvider,
+			long autoWatermarkInterval,
+			ClassLoader userCodeClassLoader,
+			boolean useMetrics) throws Exception
+	{
+		this.sourceContext = checkNotNull(sourceContext);
+		this.checkpointLock = sourceContext.getCheckpointLock();
+		this.useMetrics = useMetrics;
+		
+		// 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,
+				userCodeClassLoader);
+		
+		// if we have periodic watermarks, kick off the interval scheduler
+		if (timestampWatermarkMode == PERIODIC_WATERMARKS) {
+			KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[] parts = 
+					(KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[]) allPartitions;
+			
+			PeriodicWatermarkEmitter periodicEmitter = 
+					new PeriodicWatermarkEmitter(parts, sourceContext, processingTimeProvider, autoWatermarkInterval);
+			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). The given offsets are the internal checkpointed offsets, representing
+	 * the last processed record of each partition. Version-specific implementations of this method
+	 * need to hold the contract that the given offsets must be incremented by 1 before
+	 * committing them, so that committed offsets to Kafka represent "the next record to process".
+	 * 
+	 * @param offsets The offsets to commit to Kafka (implementations must increment offsets by 1 before committing).
+	 * @throws Exception This method forwards exceptions.
+	 */
+	public abstract void commitInternalOffsetsToKafka(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()) {
+			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
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Emits a record without attaching an existing timestamp to it.
+	 * 
+	 * <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 of the record
+	 */
+	protected void emitRecord(T record, KafkaTopicPartitionState<KPH> partitionState, long offset) throws Exception {
+		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, Long.MIN_VALUE);
+		}
+		else {
+			emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, Long.MIN_VALUE);
+		}
+	}
+
+	/**
+	 * Emits a record attaching a timestamp to it.
+	 *
+	 * <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 of the record
+	 */
+	protected void emitRecordWithTimestamp(
+			T record, KafkaTopicPartitionState<KPH> partitionState, long offset, long timestamp) throws Exception {
+
+		if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS) {
+			// fast path logic, in case there are no watermarks generated in the fetcher
+
+			// emit the record, using the checkpoint lock to guarantee
+			// atomicity of record emission and offset state update
+			synchronized (checkpointLock) {
+				sourceContext.collectWithTimestamp(record, timestamp);
+				partitionState.setOffset(offset);
+			}
+		}
+		else if (timestampWatermarkMode == PERIODIC_WATERMARKS) {
+			emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset, timestamp);
+		}
+		else {
+			emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, timestamp);
+		}
+	}
+
+	/**
+	 * Record emission, if a timestamp will be attached from an assigner that is
+	 * also a periodic watermark generator.
+	 */
+	protected void emitRecordWithTimestampAndPeriodicWatermark(
+			T record, KafkaTopicPartitionState<KPH> partitionState, long offset, long kafkaEventTimestamp)
+	{
+		@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, kafkaEventTimestamp);
+		}
+
+		// 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.
+	 */
+	protected void emitRecordWithTimestampAndPunctuatedWatermark(
+			T record, KafkaTopicPartitionState<KPH> partitionState, long offset, long kafkaEventTimestamp)
+	{
+		@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, kafkaEventTimestamp);
+		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
+	{
+		switch (timestampWatermarkMode) {
+			
+			case NO_TIMESTAMPS_WATERMARKS: {
+				@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);
+					partitions[pos++] = new KafkaTopicPartitionState<>(partition, kafkaHandle);
+				}
+
+				return partitions;
+			}
+
+			case PERIODIC_WATERMARKS: {
+				@SuppressWarnings("unchecked")
+				KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH>[] partitions =
+						(KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH>[])
+								new KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[assignedPartitions.size()];
+
+				int pos = 0;
+				for (KafkaTopicPartition partition : assignedPartitions) {
+					KPH kafkaHandle = createKafkaPartitionHandle(partition);
+
+					AssignerWithPeriodicWatermarks<T> assignerInstance =
+							watermarksPeriodic.deserializeValue(userCodeClassLoader);
+					
+					partitions[pos++] = new KafkaTopicPartitionStateWithPeriodicWatermarks<>(
+							partition, kafkaHandle, assignerInstance);
+				}
+
+				return partitions;
+			}
+
+			case PUNCTUATED_WATERMARKS: {
+				@SuppressWarnings("unchecked")
+				KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH>[] partitions =
+						(KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH>[])
+								new KafkaTopicPartitionStateWithPunctuatedWatermarks<?, ?>[assignedPartitions.size()];
+
+				int pos = 0;
+				for (KafkaTopicPartition partition : assignedPartitions) {
+					KPH kafkaHandle = createKafkaPartitionHandle(partition);
+
+					AssignerWithPunctuatedWatermarks<T> assignerInstance =
+							watermarksPunctuated.deserializeValue(userCodeClassLoader);
+
+					partitions[pos++] = new KafkaTopicPartitionStateWithPunctuatedWatermarks<>(
+							partition, kafkaHandle, assignerInstance);
+				}
+
+				return partitions;
+			}
+			default:
+				// cannot happen, add this as a guard for the future
+				throw new RuntimeException();
+		}
+	}
+
+	// ------------------------- Metrics ----------------------------------
+
+	/**
+	 * Add current and committed offsets to metric group
+	 *
+	 * @param metricGroup The metric group to use
+	 */
+	protected void addOffsetStateGauge(MetricGroup metricGroup) {
+		// add current offsets to gage
+		MetricGroup currentOffsets = metricGroup.addGroup("current-offsets");
+		MetricGroup committedOffsets = metricGroup.addGroup("committed-offsets");
+		for (KafkaTopicPartitionState<?> ktp: subscribedPartitions()) {
+			currentOffsets.gauge(ktp.getTopic() + "-" + ktp.getPartition(), new OffsetGauge(ktp, OffsetGaugeType.CURRENT_OFFSET));
+			committedOffsets.gauge(ktp.getTopic() + "-" + ktp.getPartition(), new OffsetGauge(ktp, OffsetGaugeType.COMMITTED_OFFSET));
+		}
+	}
+
+	/**
+	 * Gauge types
+	 */
+	private enum OffsetGaugeType {
+		CURRENT_OFFSET,
+		COMMITTED_OFFSET
+	}
+
+	/**
+	 * Gauge for getting the offset of a KafkaTopicPartitionState.
+	 */
+	private static class OffsetGauge implements Gauge<Long> {
+
+		private final KafkaTopicPartitionState<?> ktp;
+		private final OffsetGaugeType gaugeType;
+
+		public OffsetGauge(KafkaTopicPartitionState<?> ktp, OffsetGaugeType gaugeType) {
+			this.ktp = ktp;
+			this.gaugeType = gaugeType;
+		}
+
+		@Override
+		public Long getValue() {
+			switch(gaugeType) {
+				case COMMITTED_OFFSET:
+					return ktp.getCommittedOffset();
+				case CURRENT_OFFSET:
+					return ktp.getOffset();
+				default:
+					throw new RuntimeException("Unknown gauge type: " + gaugeType);
+			}
+		}
+	}
+ 	// ------------------------------------------------------------------------
+	
+	/**
+	 * 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 ProcessingTimeCallback {
+
+		private final KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[] allPartitions;
+		
+		private final SourceContext<?> emitter;
+		
+		private final ProcessingTimeService timerService;
+
+		private final long interval;
+		
+		private long lastWatermarkTimestamp;
+		
+		//-------------------------------------------------
+
+		PeriodicWatermarkEmitter(
+				KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[] allPartitions,
+				SourceContext<?> emitter,
+				ProcessingTimeService timerService,
+				long autoWatermarkInterval)
+		{
+			this.allPartitions = checkNotNull(allPartitions);
+			this.emitter = checkNotNull(emitter);
+			this.timerService = checkNotNull(timerService);
+			this.interval = autoWatermarkInterval;
+			this.lastWatermarkTimestamp = Long.MIN_VALUE;
+		}
+
+		//-------------------------------------------------
+		
+		public void start() {
+			timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
+		}
+		
+		@Override
+		public void onProcessingTime(long timestamp) throws Exception {
+
+			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
+			timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java
new file mode 100644
index 0000000..c736493
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+/**
+ * A proxy that communicates exceptions between threads. Typically used if an exception
+ * from a spawned thread needs to be recognized by the "parent" (spawner) thread.
+ * 
+ * <p>The spawned thread would set the exception via {@link #reportError(Throwable)}.
+ * The parent would check (at certain points) for exceptions via {@link #checkAndThrowException()}.
+ * Optionally, the parent can pass itself in the constructor to be interrupted as soon as
+ * an exception occurs.
+ * 
+ * <pre>
+ * {@code
+ * 
+ * final ExceptionProxy errorProxy = new ExceptionProxy(Thread.currentThread());
+ * 
+ * Thread subThread = new Thread() {
+ * 
+ *     public void run() {
+ *         try {
+ *             doSomething();
+ *         } catch (Throwable t) {
+ *             errorProxy.reportError(
+ *         } finally {
+ *             doSomeCleanup();
+ *         }
+ *     }
+ * };
+ * subThread.start();
+ * 
+ * doSomethingElse();
+ * errorProxy.checkAndThrowException();
+ * 
+ * doSomethingMore();
+ * errorProxy.checkAndThrowException();
+ * 
+ * try {
+ *     subThread.join();
+ * } catch (InterruptedException e) {
+ *     errorProxy.checkAndThrowException();
+ *     // restore interrupted status, if not caused by an exception
+ *     Thread.currentThread().interrupt();
+ * }
+ * }
+ * </pre>
+ */
+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;
+
+	/**
+	 * Creates an exception proxy that interrupts the given thread upon
+	 * report of an exception. The thread to interrupt may be null.
+	 * 
+	 * @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 and interrupts the target thread,
+	 * if no other exception has occurred so far.
+	 * 
+	 * <p>The exception is only set (and the interruption is only triggered),
+	 * if no other exception was set before.
+	 * 
+	 * @param t The exception that occurred
+	 */
+	public void reportError(Throwable t) {
+		// set the exception, if it is the first (and the exception is non null)
+		if (t != null && exception.compareAndSet(null, t) && toInterrupt != null) {
+			toInterrupt.interrupt();
+		}
+	}
+
+	/**
+	 * Checks whether an exception has been set via {@link #reportError(Throwable)}.
+	 * If yes, that exception if re-thrown by this method.
+	 * 
+	 * @throws Exception This method re-throws the exception, if set.
+	 */
+	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);
+			}
+		}
+	}
+}


[49/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/resources/avro/user.avsc
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/resources/avro/user.avsc b/flink-batch-connectors/flink-avro/src/test/resources/avro/user.avsc
deleted file mode 100644
index 02c11af..0000000
--- a/flink-batch-connectors/flink-avro/src/test/resources/avro/user.avsc
+++ /dev/null
@@ -1,35 +0,0 @@
-[
-{"namespace": "org.apache.flink.api.io.avro.generated",
- "type": "record",
- "name": "Address",
- "fields": [
-     {"name": "num", "type": "int"},
-     {"name": "street", "type": "string"},
-     {"name": "city", "type": "string"},
-     {"name": "state", "type": "string"},
-     {"name": "zip", "type": "string"}
-  ]
-},
-{"namespace": "org.apache.flink.api.io.avro.generated",
- "type": "record",
- "name": "User",
- "fields": [
-     {"name": "name", "type": "string"},
-     {"name": "favorite_number",  "type": ["int", "null"]},
-     {"name": "favorite_color", "type": ["string", "null"]},
-     {"name": "type_long_test", "type": ["long", "null"]},
-     {"name": "type_double_test", "type": "double"},
-     {"name": "type_null_test", "type": ["null"]},
-     {"name": "type_bool_test", "type": ["boolean"]},
-     {"name": "type_array_string", "type" : {"type" : "array", "items" : "string"}},  
-     {"name": "type_array_boolean", "type" : {"type" : "array", "items" : "boolean"}}, 
-     {"name": "type_nullable_array", "type": ["null", {"type":"array", "items":"string"}], "default":null},
-     {"name": "type_enum", "type": {"type": "enum", "name": "Colors", "symbols" : ["RED", "GREEN", "BLUE"]}},
-     {"name": "type_map", "type": {"type": "map", "values": "long"}},
-     {"name": "type_fixed",
-                 "size": 16,
-                 "type": ["null", {"name": "Fixed16", "size": 16, "type": "fixed"}] },
-     {"name": "type_union", "type": ["null", "boolean", "long", "double"]},
-     {"name": "type_nested", "type": ["null", "Address"]}
- ]
-}]

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/resources/testdata.avro
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/resources/testdata.avro b/flink-batch-connectors/flink-avro/src/test/resources/testdata.avro
deleted file mode 100644
index 45308b9..0000000
Binary files a/flink-batch-connectors/flink-avro/src/test/resources/testdata.avro and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/pom.xml b/flink-batch-connectors/flink-hadoop-compatibility/pom.xml
deleted file mode 100644
index 8f423d9..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/pom.xml
+++ /dev/null
@@ -1,182 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-	
-	<modelVersion>4.0.0</modelVersion>
-	
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-batch-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-hadoop-compatibility_2.10</artifactId>
-	<name>flink-hadoop-compatibility</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-java</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-java</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop2</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<!-- test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-		
-	</dependencies>
-
-
-	<build>
-		<plugins>
-			<!-- activate API compatibility checks -->
-			<plugin>
-				<groupId>com.github.siom79.japicmp</groupId>
-				<artifactId>japicmp-maven-plugin</artifactId>
-			</plugin>
-			<!-- Scala Compiler -->
-			<plugin>
-				<groupId>net.alchim31.maven</groupId>
-				<artifactId>scala-maven-plugin</artifactId>
-				<version>3.1.4</version>
-				<executions>
-					<!-- Run scala compiler in the process-resources phase, so that dependencies on
-						scala classes can be resolved later in the (Java) compile phase -->
-					<execution>
-						<id>scala-compile-first</id>
-						<phase>process-resources</phase>
-						<goals>
-							<goal>compile</goal>
-						</goals>
-					</execution>
-				</executions>
-				<configuration>
-					<jvmArgs>
-						<jvmArg>-Xms128m</jvmArg>
-						<jvmArg>-Xmx512m</jvmArg>
-					</jvmArgs>
-				</configuration>
-			</plugin>
-
-			<!-- Eclipse Integration -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-eclipse-plugin</artifactId>
-				<version>2.8</version>
-				<configuration>
-					<downloadSources>true</downloadSources>
-					<projectnatures>
-						<projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
-						<projectnature>org.eclipse.jdt.core.javanature</projectnature>
-					</projectnatures>
-					<buildcommands>
-						<buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
-					</buildcommands>
-					<classpathContainers>
-						<classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
-						<classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
-					</classpathContainers>
-					<excludes>
-						<exclude>org.scala-lang:scala-library</exclude>
-						<exclude>org.scala-lang:scala-compiler</exclude>
-					</excludes>
-					<sourceIncludes>
-						<sourceInclude>**/*.scala</sourceInclude>
-						<sourceInclude>**/*.java</sourceInclude>
-					</sourceIncludes>
-				</configuration>
-			</plugin>
-
-			<!-- Adding scala source directories to build path -->
-			<plugin>
-				<groupId>org.codehaus.mojo</groupId>
-				<artifactId>build-helper-maven-plugin</artifactId>
-				<version>1.7</version>
-				<executions>
-					<!-- Add src/main/scala to eclipse build path -->
-					<execution>
-						<id>add-source</id>
-						<phase>generate-sources</phase>
-						<goals>
-							<goal>add-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>src/main/scala</source>
-							</sources>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-
-			<!-- Scala Code Style, most of the configuration done via plugin management -->
-			<plugin>
-				<groupId>org.scalastyle</groupId>
-				<artifactId>scalastyle-maven-plugin</artifactId>
-				<configuration>
-					<configLocation>${project.basedir}/../../tools/maven/scalastyle-config.xml</configLocation>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
deleted file mode 100644
index 7bcb4bf..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils;
-
-import org.apache.flink.annotation.Public;
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.InvalidTypesException;
-import org.apache.flink.api.common.typeinfo.AtomicType;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.runtime.WritableComparator;
-import org.apache.flink.api.java.typeutils.runtime.WritableSerializer;
-import org.apache.hadoop.io.Writable;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Type information for data types that extend Hadoop's {@link Writable} interface. The Writable
- * interface defines the serialization and deserialization routines for the data type.
- *
- * @param <T> The type of the class represented by this type information.
- */
-@Public
-public class WritableTypeInfo<T extends Writable> extends TypeInformation<T> implements AtomicType<T> {
-	
-	private static final long serialVersionUID = 1L;
-	
-	private final Class<T> typeClass;
-
-	@PublicEvolving
-	public WritableTypeInfo(Class<T> typeClass) {
-		this.typeClass = checkNotNull(typeClass);
-
-		checkArgument(
-			Writable.class.isAssignableFrom(typeClass) && !typeClass.equals(Writable.class),
-			"WritableTypeInfo can only be used for subclasses of %s", Writable.class.getName());
-	}
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Override
-	@PublicEvolving
-	public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) {
-		if(Comparable.class.isAssignableFrom(typeClass)) {
-			return new WritableComparator(sortOrderAscending, typeClass);
-		}
-		else {
-			throw new UnsupportedOperationException("Cannot create Comparator for "+typeClass.getCanonicalName()+". " +
-													"Class does not implement Comparable interface.");
-		}
-	}
-
-	@Override
-	@PublicEvolving
-	public boolean isBasicType() {
-		return false;
-	}
-
-	@Override
-	@PublicEvolving
-	public boolean isTupleType() {
-		return false;
-	}
-
-	@Override
-	@PublicEvolving
-	public int getArity() {
-		return 1;
-	}
-	
-	@Override
-	@PublicEvolving
-	public int getTotalFields() {
-		return 1;
-	}
-
-	@Override
-	@PublicEvolving
-	public Class<T> getTypeClass() {
-		return this.typeClass;
-	}
-
-	@Override
-	@PublicEvolving
-	public boolean isKeyType() {
-		return Comparable.class.isAssignableFrom(typeClass);
-	}
-
-	@Override
-	@PublicEvolving
-	public TypeSerializer<T> createSerializer(ExecutionConfig executionConfig) {
-		return new WritableSerializer<T>(typeClass);
-	}
-	
-	@Override
-	public String toString() {
-		return "WritableType<" + typeClass.getName() + ">";
-	}	
-	
-	@Override
-	public int hashCode() {
-		return typeClass.hashCode();
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof WritableTypeInfo) {
-			@SuppressWarnings("unchecked")
-			WritableTypeInfo<T> writableTypeInfo = (WritableTypeInfo<T>) obj;
-
-			return writableTypeInfo.canEqual(this) &&
-				typeClass == writableTypeInfo.typeClass;
-
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return obj instanceof WritableTypeInfo;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-
-	@PublicEvolving
-	static <T extends Writable> TypeInformation<T> getWritableTypeInfo(Class<T> typeClass) {
-		if (Writable.class.isAssignableFrom(typeClass) && !typeClass.equals(Writable.class)) {
-			return new WritableTypeInfo<T>(typeClass);
-		}
-		else {
-			throw new InvalidTypesException("The given class is no subclass of " + Writable.class.getName());
-		}
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
deleted file mode 100644
index 3a95d94..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import com.esotericsoftware.kryo.Kryo;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.types.NormalizableKey;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.hadoop.io.Writable;
-import org.objenesis.strategy.StdInstantiatorStrategy;
-
-import java.io.IOException;
-
-public class WritableComparator<T extends Writable & Comparable<T>> extends TypeComparator<T> {
-	
-	private static final long serialVersionUID = 1L;
-	
-	private Class<T> type;
-	
-	private final boolean ascendingComparison;
-	
-	private transient T reference;
-	
-	private transient T tempReference;
-	
-	private transient Kryo kryo;
-
-	@SuppressWarnings("rawtypes")
-	private final TypeComparator[] comparators = new TypeComparator[] {this};
-
-	public WritableComparator(boolean ascending, Class<T> type) {
-		this.type = type;
-		this.ascendingComparison = ascending;
-	}
-	
-	@Override
-	public int hash(T record) {
-		return record.hashCode();
-	}
-	
-	@Override
-	public void setReference(T toCompare) {
-		checkKryoInitialized();
-
-		reference = KryoUtils.copy(toCompare, kryo, new WritableSerializer<T>(type));
-	}
-	
-	@Override
-	public boolean equalToReference(T candidate) {
-		return candidate.equals(reference);
-	}
-	
-	@Override
-	public int compareToReference(TypeComparator<T> referencedComparator) {
-		T otherRef = ((WritableComparator<T>) referencedComparator).reference;
-		int comp = otherRef.compareTo(reference);
-		return ascendingComparison ? comp : -comp;
-	}
-	
-	@Override
-	public int compare(T first, T second) {
-		int comp = first.compareTo(second);
-		return ascendingComparison ? comp : -comp;
-	}
-	
-	@Override
-	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
-		ensureReferenceInstantiated();
-		ensureTempReferenceInstantiated();
-		
-		reference.readFields(firstSource);
-		tempReference.readFields(secondSource);
-		
-		int comp = reference.compareTo(tempReference);
-		return ascendingComparison ? comp : -comp;
-	}
-	
-	@Override
-	public boolean supportsNormalizedKey() {
-		return NormalizableKey.class.isAssignableFrom(type);
-	}
-	
-	@Override
-	public int getNormalizeKeyLen() {
-		ensureReferenceInstantiated();
-		
-		NormalizableKey<?> key = (NormalizableKey<?>) reference;
-		return key.getMaxNormalizedKeyLen();
-	}
-	
-	@Override
-	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
-		return keyBytes < getNormalizeKeyLen();
-	}
-	
-	@Override
-	public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
-		NormalizableKey<?> key = (NormalizableKey<?>) record;
-		key.copyNormalizedKey(target, offset, numBytes);
-	}
-	
-	@Override
-	public boolean invertNormalizedKey() {
-		return !ascendingComparison;
-	}
-	
-	@Override
-	public TypeComparator<T> duplicate() {
-		return new WritableComparator<T>(ascendingComparison, type);
-	}
-
-	@Override
-	public int extractKeys(Object record, Object[] target, int index) {
-		target[index] = record;
-		return 1;
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public TypeComparator[] getFlatComparators() {
-		return comparators;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// unsupported normalization
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public boolean supportsSerializationWithKeyNormalization() {
-		return false;
-	}
-	
-	@Override
-	public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
-		throw new UnsupportedOperationException();
-	}
-	
-	@Override
-	public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
-		throw new UnsupportedOperationException();
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	private void checkKryoInitialized() {
-		if (this.kryo == null) {
-			this.kryo = new Kryo();
-
-			Kryo.DefaultInstantiatorStrategy instantiatorStrategy = new Kryo.DefaultInstantiatorStrategy();
-			instantiatorStrategy.setFallbackInstantiatorStrategy(new StdInstantiatorStrategy());
-			kryo.setInstantiatorStrategy(instantiatorStrategy);
-
-			this.kryo.setAsmEnabled(true);
-			this.kryo.register(type);
-		}
-	}
-	
-	private void ensureReferenceInstantiated() {
-		if (reference == null) {
-			reference = InstantiationUtil.instantiate(type, Writable.class);
-		}
-	}
-	
-	private void ensureTempReferenceInstantiated() {
-		if (tempReference == null) {
-			tempReference = InstantiationUtil.instantiate(type, Writable.class);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
deleted file mode 100644
index 9036d75..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-
-import com.esotericsoftware.kryo.Kryo;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-import org.objenesis.strategy.StdInstantiatorStrategy;
-
-import java.io.IOException;
-
-public class WritableSerializer<T extends Writable> extends TypeSerializer<T> {
-	
-	private static final long serialVersionUID = 1L;
-	
-	private final Class<T> typeClass;
-	
-	private transient Kryo kryo;
-	
-	private transient T copyInstance;
-	
-	public WritableSerializer(Class<T> typeClass) {
-		this.typeClass = typeClass;
-	}
-	
-	@SuppressWarnings("unchecked")
-	@Override
-	public T createInstance() {
-		if(typeClass == NullWritable.class) {
-			return (T) NullWritable.get();
-		}
-		return InstantiationUtil.instantiate(typeClass);
-	}
-
-
-	
-	@Override
-	public T copy(T from) {
-		checkKryoInitialized();
-
-		return KryoUtils.copy(from, kryo, this);
-	}
-	
-	@Override
-	public T copy(T from, T reuse) {
-		checkKryoInitialized();
-
-		return KryoUtils.copy(from, reuse, kryo, this);
-	}
-	
-	@Override
-	public int getLength() {
-		return -1;
-	}
-	
-	@Override
-	public void serialize(T record, DataOutputView target) throws IOException {
-		record.write(target);
-	}
-	
-	@Override
-	public T deserialize(DataInputView source) throws IOException {
-		return deserialize(createInstance(), source);
-	}
-	
-	@Override
-	public T deserialize(T reuse, DataInputView source) throws IOException {
-		reuse.readFields(source);
-		return reuse;
-	}
-	
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		ensureInstanceInstantiated();
-		copyInstance.readFields(source);
-		copyInstance.write(target);
-	}
-	
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-	
-	@Override
-	public WritableSerializer<T> duplicate() {
-		return new WritableSerializer<T>(typeClass);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	private void ensureInstanceInstantiated() {
-		if (copyInstance == null) {
-			copyInstance = createInstance();
-		}
-	}
-	
-	private void checkKryoInitialized() {
-		if (this.kryo == null) {
-			this.kryo = new Kryo();
-
-			Kryo.DefaultInstantiatorStrategy instantiatorStrategy = new Kryo.DefaultInstantiatorStrategy();
-			instantiatorStrategy.setFallbackInstantiatorStrategy(new StdInstantiatorStrategy());
-			kryo.setInstantiatorStrategy(instantiatorStrategy);
-
-			this.kryo.setAsmEnabled(true);
-			this.kryo.register(typeClass);
-		}
-	}
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public int hashCode() {
-		return this.typeClass.hashCode();
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof WritableSerializer) {
-			WritableSerializer<?> other = (WritableSerializer<?>) obj;
-
-			return other.canEqual(this) && typeClass == other.typeClass;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return obj instanceof WritableSerializer;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopInputs.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopInputs.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopInputs.java
deleted file mode 100644
index 9e8a3e4..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopInputs.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hadoopcompatibility;
-
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapreduce.Job;
-
-import java.io.IOException;
-
-/**
- * HadoopInputs is a utility class to use Apache Hadoop InputFormats with Apache Flink.
- *
- * It provides methods to create Flink InputFormat wrappers for Hadoop {@link org.apache.hadoop.mapred.InputFormat}
- * and {@link org.apache.hadoop.mapreduce.InputFormat}.
- *
- * Key value pairs produced by the Hadoop InputFormats are converted into Flink
- * {@link org.apache.flink.api.java.tuple.Tuple2 Tuple2} objects where the first field
- * ({@link org.apache.flink.api.java.tuple.Tuple2#f0 Tuple2.f0}) is the key and the second field
- * ({@link org.apache.flink.api.java.tuple.Tuple2#f1 Tuple2.f1}) is the value.
- *
- */
-
-public final class HadoopInputs {
-	// ----------------------------------- Hadoop Input Format ---------------------------------------
-
-	/**
-	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapred.FileInputFormat}.
-	 *
-	 * @return A Flink InputFormat that wraps the Hadoop FileInputFormat.
-	 */
-	public static <K,V> HadoopInputFormat<K, V> readHadoopFile(org.apache.hadoop.mapred.FileInputFormat<K,V> mapredInputFormat, Class<K> key, Class<V> value, String inputPath, JobConf job) {
-		// set input path in JobConf
-		org.apache.hadoop.mapred.FileInputFormat.addInputPath(job, new org.apache.hadoop.fs.Path(inputPath));
-		// return wrapping InputFormat
-		return createHadoopInput(mapredInputFormat, key, value, job);
-	}
-
-	/**
-	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapred.FileInputFormat}.
-	 *
-	 * @return A Flink InputFormat that wraps the Hadoop FileInputFormat.
-	 */
-	public static <K,V> HadoopInputFormat<K, V> readHadoopFile(org.apache.hadoop.mapred.FileInputFormat<K,V> mapredInputFormat, Class<K> key, Class<V> value, String inputPath) {
-		return readHadoopFile(mapredInputFormat, key, value, inputPath, new JobConf());
-	}
-
-	/**
-	 * Creates a Flink {@link InputFormat} to read a Hadoop sequence file for the given key and value classes.
-	 *
-	 * @return A Flink InputFormat that wraps a Hadoop SequenceFileInputFormat.
-	 */
-	public static <K,V> HadoopInputFormat<K, V> readSequenceFile(Class<K> key, Class<V> value, String inputPath) throws IOException {
-		return readHadoopFile(new org.apache.hadoop.mapred.SequenceFileInputFormat<K, V>(), key, value, inputPath);
-	}
-
-	/**
-	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapred.InputFormat}.
-	 *
-	 * @return A Flink InputFormat that wraps the Hadoop InputFormat.
-	 */
-	public static <K,V> HadoopInputFormat<K, V> createHadoopInput(org.apache.hadoop.mapred.InputFormat<K,V> mapredInputFormat, Class<K> key, Class<V> value, JobConf job) {
-		return new HadoopInputFormat<>(mapredInputFormat, key, value, job);
-	}
-
-	/**
-	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}.
-	 *
-	 * @return A Flink InputFormat that wraps the Hadoop FileInputFormat.
-	 */
-	public static <K,V> org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat<K, V> readHadoopFile(
-			org.apache.hadoop.mapreduce.lib.input.FileInputFormat<K,V> mapreduceInputFormat, Class<K> key, Class<V> value, String inputPath, Job job) throws IOException
-	{
-		// set input path in Job
-		org.apache.hadoop.mapreduce.lib.input.FileInputFormat.addInputPath(job, new org.apache.hadoop.fs.Path(inputPath));
-		// return wrapping InputFormat
-		return createHadoopInput(mapreduceInputFormat, key, value, job);
-	}
-
-	/**
-	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}.
-	 *
-	 * @return A Flink InputFormat that wraps the Hadoop FileInputFormat.
-	 */
-	public static <K,V> org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat<K, V> readHadoopFile(
-			org.apache.hadoop.mapreduce.lib.input.FileInputFormat<K,V> mapreduceInputFormat, Class<K> key, Class<V> value, String inputPath) throws IOException
-	{
-		return readHadoopFile(mapreduceInputFormat, key, value, inputPath, Job.getInstance());
-	}
-
-	/**
-	 * Creates a Flink {@link InputFormat} that wraps the given Hadoop {@link org.apache.hadoop.mapreduce.InputFormat}.
-	 *
-	 * @return A Flink InputFormat that wraps the Hadoop InputFormat.
-	 */
-	public static <K,V> org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat<K, V> createHadoopInput(
-			org.apache.hadoop.mapreduce.InputFormat<K,V> mapreduceInputFormat, Class<K> key, Class<V> value, Job job)
-	{
-		return new org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat<>(mapreduceInputFormat, key, value, job);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java
deleted file mode 100644
index 97ca329..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hadoopcompatibility;
-
-import org.apache.commons.cli.Option;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.hadoop.util.GenericOptionsParser;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Utility class to work with Apache Hadoop libraries.
- */
-public class HadoopUtils {
-	/**
-	 * Returns {@link ParameterTool} for the arguments parsed by {@link GenericOptionsParser}
-	 *
-	 * @param args Input array arguments. It should be parsable by {@link GenericOptionsParser}
-	 * @return A {@link ParameterTool}
-	 * @throws IOException If arguments cannot be parsed by {@link GenericOptionsParser}
-	 * @see GenericOptionsParser
-	 */
-	public static ParameterTool paramsFromGenericOptionsParser(String[] args) throws IOException {
-		Option[] options = new GenericOptionsParser(args).getCommandLine().getOptions();
-		Map<String, String> map = new HashMap<String, String>();
-		for (Option option : options) {
-			String[] split = option.getValue().split("=");
-			map.put(split[0], split[1]);
-		}
-		return ParameterTool.fromMap(map);
-	}
-}
-
-

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
deleted file mode 100644
index ba8aa90..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hadoopcompatibility.mapred;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-
-import org.apache.flink.annotation.Public;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-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.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
-import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This wrapper maps a Hadoop Mapper (mapred API) to a Flink FlatMapFunction. 
- */
-@SuppressWarnings("rawtypes")
-@Public
-public final class HadoopMapFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT> 
-					extends RichFlatMapFunction<Tuple2<KEYIN,VALUEIN>, Tuple2<KEYOUT,VALUEOUT>> 
-					implements ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private transient Mapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT> mapper;
-	private transient JobConf jobConf;
-
-	private transient HadoopOutputCollector<KEYOUT,VALUEOUT> outputCollector;
-	private transient Reporter reporter;
-	
-	/**
-	 * Maps a Hadoop Mapper (mapred API) to a Flink FlatMapFunction.
-	 * 
-	 * @param hadoopMapper The Hadoop Mapper to wrap.
-	 */
-	public HadoopMapFunction(Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopMapper) {
-		this(hadoopMapper, new JobConf());
-	}
-	
-	/**
-	 * Maps a Hadoop Mapper (mapred API) to a Flink FlatMapFunction.
-	 * The Hadoop Mapper is configured with the provided JobConf.
-	 * 
-	 * @param hadoopMapper The Hadoop Mapper to wrap.
-	 * @param conf The JobConf that is used to configure the Hadoop Mapper.
-	 */
-	public HadoopMapFunction(Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopMapper, JobConf conf) {
-		if(hadoopMapper == null) {
-			throw new NullPointerException("Mapper may not be null.");
-		}
-		if(conf == null) {
-			throw new NullPointerException("JobConf may not be null.");
-		}
-		
-		this.mapper = hadoopMapper;
-		this.jobConf = conf;
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		this.mapper.configure(jobConf);
-		
-		this.reporter = new HadoopDummyReporter();
-		this.outputCollector = new HadoopOutputCollector<KEYOUT, VALUEOUT>();
-	}
-
-	@Override
-	public void flatMap(final Tuple2<KEYIN,VALUEIN> value, final Collector<Tuple2<KEYOUT,VALUEOUT>> out) 
-			throws Exception {
-		outputCollector.setFlinkCollector(out);
-		mapper.map(value.f0, value.f1, outputCollector, reporter);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public TypeInformation<Tuple2<KEYOUT,VALUEOUT>> getProducedType() {	
-		Class<KEYOUT> outKeyClass = (Class<KEYOUT>) TypeExtractor.getParameterType(Mapper.class, mapper.getClass(), 2);
-		Class<VALUEOUT> outValClass = (Class<VALUEOUT>)TypeExtractor.getParameterType(Mapper.class, mapper.getClass(), 3);
-		
-		final TypeInformation<KEYOUT> keyTypeInfo = TypeExtractor.getForClass((Class<KEYOUT>) outKeyClass);
-		final TypeInformation<VALUEOUT> valueTypleInfo = TypeExtractor.getForClass((Class<VALUEOUT>) outValClass);
-		return new TupleTypeInfo<Tuple2<KEYOUT,VALUEOUT>>(keyTypeInfo, valueTypleInfo);
-	}
-	
-	/**
-	 * Custom serialization methods.
-	 * @see <a href="http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html">http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html</a>
-	 */
-	private void writeObject(final ObjectOutputStream out) throws IOException {
-		out.writeObject(mapper.getClass());
-		jobConf.write(out);
-	}
-
-	@SuppressWarnings("unchecked")
-	private void readObject(final ObjectInputStream in) throws IOException, ClassNotFoundException {
-		Class<Mapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT>> mapperClass = 
-				(Class<Mapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT>>)in.readObject();
-		mapper = InstantiationUtil.instantiate(mapperClass);
-		
-		jobConf = new JobConf();
-		jobConf.readFields(in);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
deleted file mode 100644
index c1acc2b..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hadoopcompatibility.mapred;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-
-import org.apache.flink.annotation.Public;
-import org.apache.flink.api.common.functions.GroupCombineFunction;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
-import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector;
-import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This wrapper maps a Hadoop Reducer and Combiner (mapred API) to a combinable Flink GroupReduceFunction.
- */
-@SuppressWarnings("rawtypes")
-@Public
-public final class HadoopReduceCombineFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT> 
-	extends RichGroupReduceFunction<Tuple2<KEYIN,VALUEIN>,Tuple2<KEYOUT,VALUEOUT>>
-	implements GroupCombineFunction<Tuple2<KEYIN,VALUEIN>, Tuple2<KEYIN,VALUEIN>>,
-				ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private transient Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> reducer;
-	private transient Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN> combiner;
-	private transient JobConf jobConf;
-	
-	private transient HadoopTupleUnwrappingIterator<KEYIN, VALUEIN> valueIterator;
-	private transient HadoopOutputCollector<KEYOUT,VALUEOUT> reduceCollector;
-	private transient HadoopOutputCollector<KEYIN,VALUEIN> combineCollector;
-	private transient Reporter reporter;
-
-	/**
-	 * Maps two Hadoop Reducer (mapred API) to a combinable Flink GroupReduceFunction.
-	 * 
-	 * @param hadoopReducer The Hadoop Reducer that is mapped to a GroupReduceFunction.
-	 * @param hadoopCombiner The Hadoop Reducer that is mapped to the combiner function.
-	 */
-	public HadoopReduceCombineFunction(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopReducer,
-										Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN> hadoopCombiner) {
-		this(hadoopReducer, hadoopCombiner, new JobConf());
-	}
-	
-	/**
-	 * Maps two Hadoop Reducer (mapred API) to a combinable Flink GroupReduceFunction.
-	 * 
-	 * @param hadoopReducer The Hadoop Reducer that is mapped to a GroupReduceFunction.
-	 * @param hadoopCombiner The Hadoop Reducer that is mapped to the combiner function.
-	 * @param conf The JobConf that is used to configure both Hadoop Reducers.
-	 */
-	public HadoopReduceCombineFunction(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopReducer,
-								Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN> hadoopCombiner, JobConf conf) {
-		if(hadoopReducer == null) {
-			throw new NullPointerException("Reducer may not be null.");
-		}
-		if(hadoopCombiner == null) {
-			throw new NullPointerException("Combiner may not be null.");
-		}
-		if(conf == null) {
-			throw new NullPointerException("JobConf may not be null.");
-		}
-		
-		this.reducer = hadoopReducer;
-		this.combiner = hadoopCombiner;
-		this.jobConf = conf;
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		this.reducer.configure(jobConf);
-		this.combiner.configure(jobConf);
-		
-		this.reporter = new HadoopDummyReporter();
-		Class<KEYIN> inKeyClass = (Class<KEYIN>) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 0);
-		TypeSerializer<KEYIN> keySerializer = TypeExtractor.getForClass(inKeyClass).createSerializer(getRuntimeContext().getExecutionConfig());
-		this.valueIterator = new HadoopTupleUnwrappingIterator<>(keySerializer);
-		this.combineCollector = new HadoopOutputCollector<>();
-		this.reduceCollector = new HadoopOutputCollector<>();
-	}
-
-	@Override
-	public void reduce(final Iterable<Tuple2<KEYIN,VALUEIN>> values, final Collector<Tuple2<KEYOUT,VALUEOUT>> out)
-			throws Exception {
-		reduceCollector.setFlinkCollector(out);
-		valueIterator.set(values.iterator());
-		reducer.reduce(valueIterator.getCurrentKey(), valueIterator, reduceCollector, reporter);
-	}
-
-	@Override
-	public void combine(final Iterable<Tuple2<KEYIN,VALUEIN>> values, final Collector<Tuple2<KEYIN,VALUEIN>> out) throws Exception {
-		combineCollector.setFlinkCollector(out);
-		valueIterator.set(values.iterator());
-		combiner.reduce(valueIterator.getCurrentKey(), valueIterator, combineCollector, reporter);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public TypeInformation<Tuple2<KEYOUT,VALUEOUT>> getProducedType() {
-		Class<KEYOUT> outKeyClass = (Class<KEYOUT>) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 2);
-		Class<VALUEOUT> outValClass = (Class<VALUEOUT>)TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 3);
-
-		final TypeInformation<KEYOUT> keyTypeInfo = TypeExtractor.getForClass(outKeyClass);
-		final TypeInformation<VALUEOUT> valueTypleInfo = TypeExtractor.getForClass(outValClass);
-		return new TupleTypeInfo<>(keyTypeInfo, valueTypleInfo);
-	}
-
-	/**
-	 * Custom serialization methods.
-	 * @see <a href="http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html">http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html</a>
-	 */
-	private void writeObject(final ObjectOutputStream out) throws IOException {
-		
-		out.writeObject(reducer.getClass());
-		out.writeObject(combiner.getClass());
-		jobConf.write(out);
-	}
-
-	@SuppressWarnings("unchecked")
-	private void readObject(final ObjectInputStream in) throws IOException, ClassNotFoundException {
-		
-		Class<Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT>> reducerClass = 
-				(Class<Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT>>)in.readObject();
-		reducer = InstantiationUtil.instantiate(reducerClass);
-		
-		Class<Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN>> combinerClass = 
-				(Class<Reducer<KEYIN,VALUEIN,KEYIN,VALUEIN>>)in.readObject();
-		combiner = InstantiationUtil.instantiate(combinerClass);
-		
-		jobConf = new JobConf();
-		jobConf.readFields(in);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
deleted file mode 100644
index 55aea24..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hadoopcompatibility.mapred;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-
-import org.apache.flink.annotation.Public;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
-import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector;
-import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This wrapper maps a Hadoop Reducer (mapred API) to a non-combinable Flink GroupReduceFunction. 
- */
-@SuppressWarnings("rawtypes")
-@Public
-public final class HadoopReduceFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT> 
-					extends RichGroupReduceFunction<Tuple2<KEYIN,VALUEIN>,Tuple2<KEYOUT,VALUEOUT>> 
-					implements ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private transient Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> reducer;
-	private transient JobConf jobConf;
-	
-	private transient HadoopTupleUnwrappingIterator<KEYIN, VALUEIN> valueIterator;
-	private transient HadoopOutputCollector<KEYOUT,VALUEOUT> reduceCollector;
-	private transient Reporter reporter;
-	
-	/**
-	 * Maps a Hadoop Reducer (mapred API) to a non-combinable Flink GroupReduceFunction.
- 	 * 
-	 * @param hadoopReducer The Hadoop Reducer to wrap.
-	 */
-	public HadoopReduceFunction(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopReducer) {
-		this(hadoopReducer, new JobConf());
-	}
-	
-	/**
-	 * Maps a Hadoop Reducer (mapred API) to a non-combinable Flink GroupReduceFunction.
- 	 * 
-	 * @param hadoopReducer The Hadoop Reducer to wrap.
-	 * @param conf The JobConf that is used to configure the Hadoop Reducer.
-	 */
-	public HadoopReduceFunction(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> hadoopReducer, JobConf conf) {
-		if(hadoopReducer == null) {
-			throw new NullPointerException("Reducer may not be null.");
-		}
-		if(conf == null) {
-			throw new NullPointerException("JobConf may not be null.");
-		}
-		
-		this.reducer = hadoopReducer;
-		this.jobConf = conf;
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		this.reducer.configure(jobConf);
-		
-		this.reporter = new HadoopDummyReporter();
-		this.reduceCollector = new HadoopOutputCollector<KEYOUT, VALUEOUT>();
-		Class<KEYIN> inKeyClass = (Class<KEYIN>) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 0);
-		TypeSerializer<KEYIN> keySerializer = TypeExtractor.getForClass(inKeyClass).createSerializer(getRuntimeContext().getExecutionConfig());
-		this.valueIterator = new HadoopTupleUnwrappingIterator<KEYIN, VALUEIN>(keySerializer);
-	}
-
-	@Override
-	public void reduce(final Iterable<Tuple2<KEYIN,VALUEIN>> values, final Collector<Tuple2<KEYOUT,VALUEOUT>> out)
-			throws Exception {
-		
-		reduceCollector.setFlinkCollector(out);
-		valueIterator.set(values.iterator());
-		reducer.reduce(valueIterator.getCurrentKey(), valueIterator, reduceCollector, reporter);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public TypeInformation<Tuple2<KEYOUT,VALUEOUT>> getProducedType() {
-		Class<KEYOUT> outKeyClass = (Class<KEYOUT>) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 2);
-		Class<VALUEOUT> outValClass = (Class<VALUEOUT>)TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 3);
-
-		final TypeInformation<KEYOUT> keyTypeInfo = TypeExtractor.getForClass((Class<KEYOUT>) outKeyClass);
-		final TypeInformation<VALUEOUT> valueTypleInfo = TypeExtractor.getForClass((Class<VALUEOUT>) outValClass);
-		return new TupleTypeInfo<Tuple2<KEYOUT,VALUEOUT>>(keyTypeInfo, valueTypleInfo);
-	}
-
-	/**
-	 * Custom serialization methods
-	 * @see <a href="http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html">http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html</a>
-	 */
-	private void writeObject(final ObjectOutputStream out) throws IOException {
-		
-		out.writeObject(reducer.getClass());
-		jobConf.write(out);		
-	}
-
-	@SuppressWarnings("unchecked")
-	private void readObject(final ObjectInputStream in) throws IOException, ClassNotFoundException {
-		
-		Class<Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT>> reducerClass = 
-				(Class<Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT>>)in.readObject();
-		reducer = InstantiationUtil.instantiate(reducerClass);
-		
-		jobConf = new JobConf();
-		jobConf.readFields(in);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopOutputCollector.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopOutputCollector.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopOutputCollector.java
deleted file mode 100644
index bfe03d3..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopOutputCollector.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hadoopcompatibility.mapred.wrapper;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-import org.apache.hadoop.mapred.OutputCollector;
-
-import java.io.IOException;
-
-/**
- * A Hadoop OutputCollector that wraps a Flink OutputCollector.
- * On each call of collect() the data is forwarded to the wrapped Flink collector.
- */
-public final class HadoopOutputCollector<KEY,VALUE> implements OutputCollector<KEY,VALUE> {
-
-	private Collector<Tuple2<KEY,VALUE>> flinkCollector;
-
-	private final Tuple2<KEY,VALUE> outTuple = new Tuple2<KEY, VALUE>();
-
-	/**
-	 * Set the wrapped Flink collector.
-	 * 
-	 * @param flinkCollector The wrapped Flink OutputCollector.
-	 */
-	public void setFlinkCollector(Collector<Tuple2<KEY, VALUE>> flinkCollector) {
-		this.flinkCollector = flinkCollector;
-	}
-	
-	/**
-	 * Use the wrapped Flink collector to collect a key-value pair for Flink. 
-	 * 
-	 * @param key the key to collect
-	 * @param val the value to collect
-	 * @throws IOException unexpected of key or value in key-value pair.
-	 */
-	@Override
-	public void collect(final KEY key, final VALUE val) throws IOException {
-		this.outTuple.f0 = key;
-		this.outTuple.f1 = val;
-		this.flinkCollector.collect(outTuple);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java
deleted file mode 100644
index 2d204b8..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hadoopcompatibility.mapred.wrapper;
-
-import java.util.Iterator;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.operators.translation.TupleUnwrappingIterator;
-import org.apache.flink.api.java.tuple.Tuple2;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Wraps a Flink Tuple2 (key-value-pair) iterator into an iterator over the second (value) field.
- */
-public class HadoopTupleUnwrappingIterator<KEY,VALUE> 
-		extends TupleUnwrappingIterator<VALUE, KEY> implements java.io.Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private final TypeSerializer<KEY> keySerializer;
-
-	private transient Iterator<Tuple2<KEY,VALUE>> iterator;
-	
-	private transient KEY curKey;
-	private transient VALUE firstValue;
-	private transient boolean atFirst;
-
-	public HadoopTupleUnwrappingIterator(TypeSerializer<KEY> keySerializer) {
-		this.keySerializer = checkNotNull(keySerializer);
-	}
-	
-	/**
-	 * Set the Flink iterator to wrap.
-	 * 
-	 * @param iterator The Flink iterator to wrap.
-	 */
-	@Override
-	public void set(final Iterator<Tuple2<KEY,VALUE>> iterator) {
-		this.iterator = iterator;
-		if(this.hasNext()) {
-			final Tuple2<KEY, VALUE> tuple = iterator.next();
-			this.curKey = keySerializer.copy(tuple.f0);
-			this.firstValue = tuple.f1;
-			this.atFirst = true;
-		} else {
-			this.atFirst = false;
-		}
-	}
-	
-	@Override
-	public boolean hasNext() {
-		if(this.atFirst) {
-			return true;
-		}
-		return iterator.hasNext();
-	}
-	
-	@Override
-	public VALUE next() {
-		if(this.atFirst) {
-			this.atFirst = false;
-			return firstValue;
-		}
-		
-		final Tuple2<KEY, VALUE> tuple = iterator.next();
-		return tuple.f1;
-	}
-	
-	public KEY getCurrentKey() {
-		return this.curKey;
-	}
-	
-	@Override
-	public void remove() {
-		throw new UnsupportedOperationException();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/main/scala/org/apache/flink/hadoopcompatibility/scala/HadoopInputs.scala
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/scala/org/apache/flink/hadoopcompatibility/scala/HadoopInputs.scala b/flink-batch-connectors/flink-hadoop-compatibility/src/main/scala/org/apache/flink/hadoopcompatibility/scala/HadoopInputs.scala
deleted file mode 100644
index 133a5f4..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/scala/org/apache/flink/hadoopcompatibility/scala/HadoopInputs.scala
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.hadoopcompatibility.scala
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala.hadoop.mapreduce
-import org.apache.flink.api.scala.hadoop.mapred
-import org.apache.hadoop.fs.{Path => HadoopPath}
-import org.apache.hadoop.mapred.{JobConf, FileInputFormat => MapredFileInputFormat, InputFormat => MapredInputFormat}
-import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => MapreduceFileInputFormat}
-import org.apache.hadoop.mapreduce.{Job, InputFormat => MapreduceInputFormat}
-
-/**
-  * HadoopInputs is a utility class to use Apache Hadoop InputFormats with Apache Flink.
-  *
-  * It provides methods to create Flink InputFormat wrappers for Hadoop
-  * [[org.apache.hadoop.mapred.InputFormat]] and [[org.apache.hadoop.mapreduce.InputFormat]].
-  *
-  * Key value pairs produced by the Hadoop InputFormats are converted into [[Tuple2]] where
-  * the first field is the key and the second field is the value.
-  *
-  */
-object HadoopInputs {
-
-  /**
-    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
-    * [[org.apache.hadoop.mapred.FileInputFormat]].
-    */
-  def readHadoopFile[K, V](
-      mapredInputFormat: MapredFileInputFormat[K, V],
-      key: Class[K],
-      value: Class[V],
-      inputPath: String,
-      job: JobConf)(implicit tpe: TypeInformation[(K, V)]): mapred.HadoopInputFormat[K, V] = {
-
-    // set input path in JobConf
-    MapredFileInputFormat.addInputPath(job, new HadoopPath(inputPath))
-    // wrap mapredInputFormat
-    createHadoopInput(mapredInputFormat, key, value, job)
-  }
-
-  /**
-    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
-    * [[org.apache.hadoop.mapred.FileInputFormat]].
-    */
-  def readHadoopFile[K, V](
-      mapredInputFormat: MapredFileInputFormat[K, V],
-      key: Class[K],
-      value: Class[V],
-      inputPath: String)(implicit tpe: TypeInformation[(K, V)]): mapred.HadoopInputFormat[K, V] = {
-
-    readHadoopFile(mapredInputFormat, key, value, inputPath, new JobConf)
-  }
-
-  /**
-    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that reads a Hadoop sequence
-    * file with the given key and value classes.
-    */
-  def readSequenceFile[K, V](
-      key: Class[K],
-      value: Class[V],
-      inputPath: String)(implicit tpe: TypeInformation[(K, V)]): mapred.HadoopInputFormat[K, V] = {
-
-    readHadoopFile(
-      new org.apache.hadoop.mapred.SequenceFileInputFormat[K, V],
-      key,
-      value,
-      inputPath
-    )
-  }
-
-  /**
-    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
-    * [[org.apache.hadoop.mapred.InputFormat]].
-    */
-  def createHadoopInput[K, V](
-      mapredInputFormat: MapredInputFormat[K, V],
-      key: Class[K],
-      value: Class[V],
-      job: JobConf)(implicit tpe: TypeInformation[(K, V)]): mapred.HadoopInputFormat[K, V] = {
-
-    new mapred.HadoopInputFormat[K, V](mapredInputFormat, key, value, job)
-  }
-
-  /**
-    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
-    * [[org.apache.hadoop.mapreduce.lib.input.FileInputFormat]].
-    */
-  def readHadoopFile[K, V](
-      mapreduceInputFormat: MapreduceFileInputFormat[K, V],
-      key: Class[K],
-      value: Class[V],
-      inputPath: String,
-      job: Job)(implicit tpe: TypeInformation[(K, V)]): mapreduce.HadoopInputFormat[K, V] = {
-
-    // set input path in Job
-    MapreduceFileInputFormat.addInputPath(job, new HadoopPath(inputPath))
-    // wrap mapreduceInputFormat
-    createHadoopInput(mapreduceInputFormat, key, value, job)
-  }
-
-  /**
-    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
-    * [[org.apache.hadoop.mapreduce.lib.input.FileInputFormat]].
-    */
-  def readHadoopFile[K, V](
-      mapreduceInputFormat: MapreduceFileInputFormat[K, V],
-      key: Class[K],
-      value: Class[V],
-      inputPath: String)(implicit tpe: TypeInformation[(K, V)]): mapreduce.HadoopInputFormat[K, V] =
-  {
-    readHadoopFile(mapreduceInputFormat, key, value, inputPath, Job.getInstance)
-  }
-
-  /**
-    * Creates a Flink [[org.apache.flink.api.common.io.InputFormat]] that wraps the given Hadoop
-    * [[org.apache.hadoop.mapreduce.InputFormat]].
-    */
-  def createHadoopInput[K, V](
-      mapreduceInputFormat: MapreduceInputFormat[K, V],
-      key: Class[K],
-      value: Class[V],
-      job: Job)(implicit tpe: TypeInformation[(K, V)]): mapreduce.HadoopInputFormat[K, V] = {
-
-    new mapreduce.HadoopInputFormat[K, V](mapreduceInputFormat, key, value, job)
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableExtractionTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableExtractionTest.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableExtractionTest.java
deleted file mode 100644
index 2aefd9f..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableExtractionTest.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils;
-
-import org.apache.flink.api.common.functions.InvalidTypesException;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparator;
-
-import org.junit.Test;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-@SuppressWarnings("serial")
-public class WritableExtractionTest {
-
-	@Test
-	public void testDetectWritable() {
-		// writable interface itself must not be writable
-		assertFalse(TypeExtractor.isHadoopWritable(Writable.class));
-
-		// various forms of extension
-		assertTrue(TypeExtractor.isHadoopWritable(DirectWritable.class));
-		assertTrue(TypeExtractor.isHadoopWritable(ViaInterfaceExtension.class));
-		assertTrue(TypeExtractor.isHadoopWritable(ViaAbstractClassExtension.class));
-
-		// some non-writables
-		assertFalse(TypeExtractor.isHadoopWritable(String.class));
-		assertFalse(TypeExtractor.isHadoopWritable(List.class));
-		assertFalse(TypeExtractor.isHadoopWritable(WritableComparator.class));
-	}
-
-	@Test
-	public void testCreateWritableInfo() {
-		TypeInformation<DirectWritable> info1 =
-				TypeExtractor.createHadoopWritableTypeInfo(DirectWritable.class);
-		assertEquals(DirectWritable.class, info1.getTypeClass());
-
-		TypeInformation<ViaInterfaceExtension> info2 =
-				TypeExtractor.createHadoopWritableTypeInfo(ViaInterfaceExtension.class);
-		assertEquals(ViaInterfaceExtension.class, info2.getTypeClass());
-
-		TypeInformation<ViaAbstractClassExtension> info3 = 
-				TypeExtractor.createHadoopWritableTypeInfo(ViaAbstractClassExtension.class);
-		assertEquals(ViaAbstractClassExtension.class, info3.getTypeClass());
-	}
-
-	@Test
-	public void testValidateTypeInfo() {
-		// validate unrelated type info
-		TypeExtractor.validateIfWritable(BasicTypeInfo.STRING_TYPE_INFO, String.class);
-
-		// validate writable type info correctly
-		TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
-				DirectWritable.class), DirectWritable.class);
-		TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
-				ViaInterfaceExtension.class), ViaInterfaceExtension.class);
-		TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
-				ViaAbstractClassExtension.class), ViaAbstractClassExtension.class);
-
-		// incorrect case: not writable at all
-		try {
-			TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
-					DirectWritable.class), String.class);
-			fail("should have failed with an exception");
-		} catch (InvalidTypesException e) {
-			// expected
-		}
-
-		// incorrect case: wrong writable
-		try {
-			TypeExtractor.validateIfWritable(new WritableTypeInfo<>(
-					ViaInterfaceExtension.class), DirectWritable.class);
-			fail("should have failed with an exception");
-		} catch (InvalidTypesException e) {
-			// expected
-		}
-	}
-
-	@Test
-	public void testExtractFromFunction() {
-		RichMapFunction<DirectWritable, DirectWritable> function = new RichMapFunction<DirectWritable, DirectWritable>() {
-			@Override
-			public DirectWritable map(DirectWritable value) throws Exception {
-				return null;
-			}
-		};
-
-		TypeInformation<DirectWritable> outType = 
-				TypeExtractor.getMapReturnTypes(function, new WritableTypeInfo<>(DirectWritable.class));
-
-		assertTrue(outType instanceof WritableTypeInfo);
-		assertEquals(DirectWritable.class, outType.getTypeClass());
-	}
-
-	@Test
-	public void testExtractAsPartOfPojo() {
-		PojoTypeInfo<PojoWithWritable> pojoInfo = 
-				(PojoTypeInfo<PojoWithWritable>) TypeExtractor.getForClass(PojoWithWritable.class);
-
-		boolean foundWritable = false;
-		for (int i = 0; i < pojoInfo.getArity(); i++) {
-			PojoField field = pojoInfo.getPojoFieldAt(i);
-			String name = field.getField().getName();
-			
-			if (name.equals("hadoopCitizen")) {
-				if (foundWritable) {
-					fail("already seen");
-				}
-				foundWritable = true;
-				assertEquals(new WritableTypeInfo<>(DirectWritable.class), field.getTypeInformation());
-				assertEquals(DirectWritable.class, field.getTypeInformation().getTypeClass());
-				
-			}
-		}
-		
-		assertTrue("missed the writable type", foundWritable);
-	}
-
-	@Test
-	public void testInputValidationError() {
-
-		RichMapFunction<Writable, String> function = new RichMapFunction<Writable, String>() {
-			@Override
-			public String map(Writable value) throws Exception {
-				return null;
-			}
-		};
-
-		@SuppressWarnings("unchecked")
-		TypeInformation<Writable> inType = 
-				(TypeInformation<Writable>) (TypeInformation<?>) new WritableTypeInfo<>(DirectWritable.class);
-		
-		try {
-			TypeExtractor.getMapReturnTypes(function, inType);
-			fail("exception expected");
-		}
-		catch (InvalidTypesException e) {
-			// right
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  test type classes
-	// ------------------------------------------------------------------------
-
-	public interface ExtendedWritable extends Writable {}
-
-	public static abstract class AbstractWritable implements Writable {}
-
-	public static class DirectWritable implements Writable {
-
-		@Override
-		public void write(DataOutput dataOutput) throws IOException {}
-
-		@Override
-		public void readFields(DataInput dataInput) throws IOException {}
-	}
-
-	public static class ViaInterfaceExtension implements ExtendedWritable {
-
-		@Override
-		public void write(DataOutput dataOutput) throws IOException {}
-
-		@Override
-		public void readFields(DataInput dataInput) throws IOException {}
-	}
-
-	public static class ViaAbstractClassExtension extends AbstractWritable {
-
-		@Override
-		public void write(DataOutput dataOutput) throws IOException {}
-
-		@Override
-		public void readFields(DataInput dataInput) throws IOException {}
-	}
-
-	public static class PojoWithWritable {
-		public String str;
-		public DirectWritable hadoopCitizen;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableInfoParserTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableInfoParserTest.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableInfoParserTest.java
deleted file mode 100644
index 3d2b652..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableInfoParserTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils;
-
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.hadoop.io.Writable;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-public class WritableInfoParserTest {
-
-	@Test
-	public void testWritableType() {
-		TypeInformation<?> ti = TypeInfoParser.parse(
-				"Writable<org.apache.flink.api.java.typeutils.WritableInfoParserTest$MyWritable>");
-
-		Assert.assertTrue(ti instanceof WritableTypeInfo<?>);
-		Assert.assertEquals(MyWritable.class, ((WritableTypeInfo<?>) ti).getTypeClass());
-	}
-
-	@Test
-	public void testPojoWithWritableType() {
-		TypeInformation<?> ti = TypeInfoParser.parse(
-				"org.apache.flink.api.java.typeutils.WritableInfoParserTest$MyPojo<"
-				+ "basic=Integer,"
-				+ "tuple=Tuple2<String, Integer>,"
-				+ "hadoopCitizen=Writable<org.apache.flink.api.java.typeutils.WritableInfoParserTest$MyWritable>,"
-				+ "array=String[]"
-				+ ">");
-		Assert.assertTrue(ti instanceof PojoTypeInfo);
-		PojoTypeInfo<?> pti = (PojoTypeInfo<?>) ti;
-		Assert.assertEquals("array", pti.getPojoFieldAt(0).getField().getName());
-		Assert.assertTrue(pti.getPojoFieldAt(0).getTypeInformation() instanceof BasicArrayTypeInfo);
-		Assert.assertEquals("basic", pti.getPojoFieldAt(1).getField().getName());
-		Assert.assertTrue(pti.getPojoFieldAt(1).getTypeInformation() instanceof BasicTypeInfo);
-		Assert.assertEquals("hadoopCitizen", pti.getPojoFieldAt(2).getField().getName());
-		Assert.assertTrue(pti.getPojoFieldAt(2).getTypeInformation() instanceof WritableTypeInfo);
-		Assert.assertEquals("tuple", pti.getPojoFieldAt(3).getField().getName());
-		Assert.assertTrue(pti.getPojoFieldAt(3).getTypeInformation() instanceof TupleTypeInfo);
-	}
-	// ------------------------------------------------------------------------
-	//  Test types
-	// ------------------------------------------------------------------------
-
-	public static class MyWritable implements Writable {
-
-		@Override
-		public void write(DataOutput out) throws IOException {}
-
-		@Override
-		public void readFields(DataInput in) throws IOException {}
-	}
-
-	public static class MyPojo {
-		public Integer basic;
-		public Tuple2<String, Integer> tuple;
-		public MyWritable hadoopCitizen;
-		public String[] array;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java b/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
deleted file mode 100644
index eb9cdf0..0000000
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.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.api.java.typeutils;
-
-import org.apache.flink.util.TestLogger;
-import org.apache.hadoop.io.Writable;
-import org.junit.Test;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
-public class WritableTypeInfoTest extends TestLogger {
-	
-	@Test
-	public void testWritableTypeInfoEquality() {
-		WritableTypeInfo<TestClass> tpeInfo1 = new WritableTypeInfo<>(TestClass.class);
-		WritableTypeInfo<TestClass> tpeInfo2 = new WritableTypeInfo<>(TestClass.class);
-
-		assertEquals(tpeInfo1, tpeInfo2);
-		assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
-	}
-
-	@Test
-	public void testWritableTypeInfoInequality() {
-		WritableTypeInfo<TestClass> tpeInfo1 = new WritableTypeInfo<>(TestClass.class);
-		WritableTypeInfo<AlternateClass> tpeInfo2 = new WritableTypeInfo<>(AlternateClass.class);
-
-		assertNotEquals(tpeInfo1, tpeInfo2);
-	}
-
-	// ------------------------------------------------------------------------
-	//  test types
-	// ------------------------------------------------------------------------
-
-	public static class TestClass implements Writable {
-
-		@Override
-		public void write(DataOutput dataOutput) throws IOException {}
-
-		@Override
-		public void readFields(DataInput dataInput) throws IOException {}
-	}
-
-	public static class AlternateClass implements Writable {
-
-		@Override
-		public void write(DataOutput dataOutput) throws IOException {}
-
-		@Override
-		public void readFields(DataInput dataInput) throws IOException {}
-	}
-}


[29/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
new file mode 100644
index 0000000..dccf698
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
@@ -0,0 +1,291 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+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.source.RichParallelSourceFunction;
+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.util.InstantiationUtil;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import org.junit.ClassRule;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+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 LocalFlinkMiniCluster flink;
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	protected static Properties secureProps = new Properties();
+
+	@BeforeClass
+	public static void prepare() throws IOException, ClassNotFoundException {
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    Starting KafkaShortRetentionTestBase ");
+		LOG.info("-------------------------------------------------------------------------");
+
+		Configuration flinkConfig = new Configuration();
+
+		// dynamically load the implementation for the test
+		Class<?> clazz = Class.forName("org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl");
+		kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz);
+
+		LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion());
+
+		if(kafkaServer.isSecureRunSupported()) {
+			secureProps = kafkaServer.getSecureProperties();
+		}
+
+		Properties specificProperties = new Properties();
+		specificProperties.setProperty("log.retention.hours", "0");
+		specificProperties.setProperty("log.retention.minutes", "0");
+		specificProperties.setProperty("log.retention.ms", "250");
+		specificProperties.setProperty("log.retention.check.interval.ms", "100");
+		kafkaServer.prepare(1, specificProperties, false);
+
+		standardProps = kafkaServer.getStandardProperties();
+
+		// start also a re-usable Flink mini cluster
+		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
+		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
+
+		flink = new LocalFlinkMiniCluster(flinkConfig, false);
+		flink.start();
+	}
+
+	@AfterClass
+	public static void shutDownServices() {
+		if (flink != null) {
+			flink.shutdown();
+		}
+		kafkaServer.shutdown();
+
+		secureProps.clear();
+	}
+
+	/**
+	 * This test is concurrently reading and writing from a kafka topic.
+	 * The job will run for a while
+	 * In a special deserializationSchema, we make sure that the offsets from the topic
+	 * are non-continuous (because the data is expiring faster than its consumed --> with auto.offset.reset = 'earliest', some offsets will not show up)
+	 *
+	 */
+	private static boolean stopProducer = false;
+
+	public void runAutoOffsetResetTest() throws Exception {
+		final String topic = "auto-offset-reset-test";
+
+		final int parallelism = 1;
+		final int elementsPerPartition = 50000;
+
+		Properties tprops = new Properties();
+		tprops.setProperty("retention.ms", "250");
+		kafkaServer.createTestTopic(topic, parallelism, 1, tprops);
+
+		final StreamExecutionEnvironment env =
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flink.getLeaderRPCPort());
+		env.setParallelism(parallelism);
+		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
+		env.getConfig().disableSysoutLogging();
+
+
+		// ----------- add producer dataflow ----------
+
+
+		DataStream<String> stream = env.addSource(new RichParallelSourceFunction<String>() {
+
+			private boolean running = true;
+
+			@Override
+			public void run(SourceContext<String> ctx) throws InterruptedException {
+				int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition;
+				int limit = cnt + elementsPerPartition;
+
+
+				while (running && !stopProducer && cnt < limit) {
+					ctx.collect("element-" + cnt);
+					cnt++;
+					Thread.sleep(10);
+				}
+				LOG.info("Stopping producer");
+			}
+
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		});
+		Properties props = new Properties();
+		props.putAll(standardProps);
+		props.putAll(secureProps);
+		kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), props, null);
+
+		// ----------- add consumer dataflow ----------
+
+		NonContinousOffsetsDeserializationSchema deserSchema = new NonContinousOffsetsDeserializationSchema();
+		FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer(topic, deserSchema, props);
+
+		DataStreamSource<String> consuming = env.addSource(source);
+		consuming.addSink(new DiscardingSink<String>());
+
+		tryExecute(env, "run auto offset reset test");
+
+		kafkaServer.deleteTestTopic(topic);
+	}
+
+	
+	private class NonContinousOffsetsDeserializationSchema implements KeyedDeserializationSchema<String> {
+		private int numJumps;
+		long nextExpected = 0;
+
+		@Override
+		public String deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
+			if(offset != nextExpected) {
+				numJumps++;
+				nextExpected = offset;
+				LOG.info("Registered now jump at offset {}", offset);
+			}
+			nextExpected++;
+			try {
+				Thread.sleep(10); // slow down data consumption to trigger log eviction
+			} catch (InterruptedException e) {
+				throw new RuntimeException("Stopping it");
+			}
+			return "";
+		}
+
+		@Override
+		public boolean isEndOfStream(String nextElement) {
+			if( numJumps >= 5) {
+				// we saw 5 jumps and no failures --> consumer can handle auto.offset.reset
+				stopProducer = true;
+				return true;
+			}
+			return false;
+		}
+
+		@Override
+		public TypeInformation<String> getProducedType() {
+			return TypeInfoParser.parse("String");
+		}
+	}
+
+
+	/**
+	 * Ensure that the consumer is properly failing if "auto.offset.reset" is set to "none"
+	 * @throws Exception
+	 */
+	public void runFailOnAutoOffsetResetNone() throws Exception {
+		final String topic = "auto-offset-reset-none-test";
+		final int parallelism = 1;
+		
+		kafkaServer.createTestTopic(topic, parallelism, 1);
+
+		final StreamExecutionEnvironment env =
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flink.getLeaderRPCPort());
+		env.setParallelism(parallelism);
+		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
+		env.getConfig().disableSysoutLogging();
+		
+		// ----------- add consumer ----------
+
+		Properties customProps = new Properties();
+		customProps.putAll(standardProps);
+		customProps.putAll(secureProps);
+		customProps.setProperty("auto.offset.reset", "none"); // test that "none" leads to an exception
+		FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), customProps);
+
+		DataStreamSource<String> consuming = env.addSource(source);
+		consuming.addSink(new DiscardingSink<String>());
+
+		try {
+			env.execute("Test auto offset reset none");
+		} catch(Throwable e) {
+			System.out.println("MESSAGE: " + e.getCause().getCause().getMessage());
+			// check if correct exception has been thrown
+			if(!e.getCause().getCause().getMessage().contains("Unable to find previous offset")  // kafka 0.8
+			 && !e.getCause().getCause().getMessage().contains("Undefined offset with no reset policy for partition") // kafka 0.9
+					) {
+				throw e;
+			}
+		}
+
+		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.putAll(secureProps);
+		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/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
new file mode 100644
index 0000000..ae0af52
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.typeutils.RowTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.connectors.kafka.internals.TypeUtil;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.util.Properties;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+
+public abstract class KafkaTableSinkTestBase {
+
+	private static final String TOPIC = "testTopic";
+	protected static final String[] FIELD_NAMES = new String[] {"field1", "field2"};
+	private static final TypeInformation[] FIELD_TYPES = TypeUtil.toTypeInfo(new Class[] {Integer.class, String.class});
+	private static final KafkaPartitioner<Row> PARTITIONER = new CustomPartitioner();
+	private static final Properties PROPERTIES = createSinkProperties();
+	@SuppressWarnings("unchecked")
+	private final FlinkKafkaProducerBase<Row> PRODUCER = new FlinkKafkaProducerBase<Row>(
+		TOPIC, new KeyedSerializationSchemaWrapper(getSerializationSchema()), PROPERTIES, PARTITIONER) {
+
+		@Override
+		protected void flush() {}
+	};
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testKafkaTableSink() throws Exception {
+		DataStream dataStream = mock(DataStream.class);
+
+		KafkaTableSink kafkaTableSink = spy(createTableSink());
+		kafkaTableSink.emitDataStream(dataStream);
+
+		verify(dataStream).addSink(eq(PRODUCER));
+
+		verify(kafkaTableSink).createKafkaProducer(
+			eq(TOPIC),
+			eq(PROPERTIES),
+			any(getSerializationSchema().getClass()),
+			eq(PARTITIONER));
+	}
+
+	@Test
+	public void testConfiguration() {
+		KafkaTableSink kafkaTableSink = createTableSink();
+		KafkaTableSink newKafkaTableSink = kafkaTableSink.configure(FIELD_NAMES, FIELD_TYPES);
+		assertNotSame(kafkaTableSink, newKafkaTableSink);
+
+		assertArrayEquals(FIELD_NAMES, newKafkaTableSink.getFieldNames());
+		assertArrayEquals(FIELD_TYPES, newKafkaTableSink.getFieldTypes());
+		assertEquals(new RowTypeInfo(FIELD_TYPES), newKafkaTableSink.getOutputType());
+	}
+
+	protected abstract KafkaTableSink createTableSink(String topic, Properties properties,
+			KafkaPartitioner<Row> partitioner, FlinkKafkaProducerBase<Row> kafkaProducer);
+
+	protected abstract SerializationSchema<Row> getSerializationSchema();
+
+	private KafkaTableSink createTableSink() {
+		return createTableSink(TOPIC, PROPERTIES, PARTITIONER, PRODUCER);
+	}
+
+	private static Properties createSinkProperties() {
+		Properties properties = new Properties();
+		properties.setProperty("bootstrap.servers", "localhost:12345");
+		return properties;
+	}
+
+	private static class CustomPartitioner extends KafkaPartitioner<Row> implements Serializable {
+		@Override
+		public int partition(Row next, byte[] serializedKey, byte[] serializedValue, int numPartitions) {
+			return 0;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
new file mode 100644
index 0000000..2a281e8
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import java.util.Properties;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.junit.Test;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+
+public abstract class KafkaTableSourceTestBase {
+
+	private static final String TOPIC = "testTopic";
+	private static final String[] FIELD_NAMES = new String[] { "long", "string", "boolean", "double", "missing-field" };
+	private static final TypeInformation<?>[] FIELD_TYPES = new TypeInformation<?>[] {
+		BasicTypeInfo.LONG_TYPE_INFO,
+		BasicTypeInfo.STRING_TYPE_INFO,
+		BasicTypeInfo.BOOLEAN_TYPE_INFO,
+		BasicTypeInfo.DOUBLE_TYPE_INFO,
+		BasicTypeInfo.LONG_TYPE_INFO };
+	private static final Properties PROPERTIES = createSourceProperties();
+
+	@Test
+	public void testKafkaTableSource() {
+		KafkaTableSource kafkaTableSource = spy(createTableSource());
+		StreamExecutionEnvironment env = mock(StreamExecutionEnvironment.class);
+		kafkaTableSource.getDataStream(env);
+
+		verify(env).addSource(any(getFlinkKafkaConsumer()));
+
+		verify(kafkaTableSource).getKafkaConsumer(
+			eq(TOPIC),
+			eq(PROPERTIES),
+			any(getDeserializationSchema()));
+	}
+
+	protected abstract KafkaTableSource createTableSource(String topic, Properties properties,
+			String[] fieldNames, TypeInformation<?>[] typeInfo);
+
+	protected abstract Class<DeserializationSchema<Row>> getDeserializationSchema();
+
+	protected abstract Class<FlinkKafkaConsumerBase<Row>> getFlinkKafkaConsumer();
+
+	private KafkaTableSource createTableSource() {
+		return createTableSource(TOPIC, PROPERTIES, FIELD_NAMES, FIELD_TYPES);
+	}
+
+	private static Properties createSourceProperties() {
+		Properties properties = new Properties();
+		properties.setProperty("zookeeper.connect", "dummy");
+		properties.setProperty("group.id", "dummy");
+		return properties;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
new file mode 100644
index 0000000..5cec4f0
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.jmx.JMXReporter;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+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;
+
+import org.junit.ClassRule;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.IOException;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+
+/**
+ * The base for the Kafka tests. It brings up:
+ * <ul>
+ *     <li>A ZooKeeper mini cluster</li>
+ *     <li>Three Kafka Brokers (mini clusters)</li>
+ *     <li>A Flink mini cluster</li>
+ * </ul>
+ * 
+ * <p>Code in this test is based on the following GitHub repository:
+ * <a href="https://github.com/sakserv/hadoop-mini-clusters">
+ *   https://github.com/sakserv/hadoop-mini-clusters</a> (ASL licensed),
+ * as per commit <i>bc6b2b2d5f6424d5f377aa6c0871e82a956462ef</i></p>
+ */
+@SuppressWarnings("serial")
+public abstract class KafkaTestBase extends TestLogger {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class);
+	
+	protected static final int NUMBER_OF_KAFKA_SERVERS = 3;
+
+	protected static String brokerConnectionStrings;
+
+	protected static Properties standardProps;
+	
+	protected static LocalFlinkMiniCluster flink;
+
+	protected static int flinkPort;
+
+	protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
+
+	protected static KafkaTestEnvironment kafkaServer;
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	protected static Properties secureProps = new Properties();
+
+	// ------------------------------------------------------------------------
+	//  Setup and teardown of the mini clusters
+	// ------------------------------------------------------------------------
+	
+	@BeforeClass
+	public static void prepare() throws IOException, ClassNotFoundException {
+
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    Starting KafkaTestBase ");
+		LOG.info("-------------------------------------------------------------------------");
+
+		startClusters(false);
+
+	}
+
+	@AfterClass
+	public static void shutDownServices() {
+
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    Shut down KafkaTestBase ");
+		LOG.info("-------------------------------------------------------------------------");
+
+		shutdownClusters();
+
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    KafkaTestBase finished");
+		LOG.info("-------------------------------------------------------------------------");
+	}
+
+	protected static Configuration getFlinkConfiguration() {
+		Configuration flinkConfig = new Configuration();
+		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
+		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
+		flinkConfig.setString(ConfigConstants.METRICS_REPORTERS_LIST, "my_reporter");
+		flinkConfig.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "my_reporter." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName());
+		return flinkConfig;
+	}
+
+	protected static void startClusters(boolean secureMode) throws ClassNotFoundException {
+
+		// dynamically load the implementation for the test
+		Class<?> clazz = Class.forName("org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl");
+		kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz);
+
+		LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion());
+
+		kafkaServer.prepare(NUMBER_OF_KAFKA_SERVERS, secureMode);
+
+		standardProps = kafkaServer.getStandardProperties();
+
+		brokerConnectionStrings = kafkaServer.getBrokerConnectionString();
+
+		if (secureMode) {
+			if (!kafkaServer.isSecureRunSupported()) {
+				throw new IllegalStateException(
+					"Attempting to test in secure mode but secure mode not supported by the KafkaTestEnvironment.");
+			}
+			secureProps = kafkaServer.getSecureProperties();
+		}
+
+		// start also a re-usable Flink mini cluster
+		flink = new LocalFlinkMiniCluster(getFlinkConfiguration(), false);
+		flink.start();
+
+		flinkPort = flink.getLeaderRPCPort();
+
+	}
+
+	protected static void shutdownClusters() {
+
+		flinkPort = -1;
+		if (flink != null) {
+			flink.shutdown();
+		}
+
+		if(secureProps != null) {
+			secureProps.clear();
+		}
+
+		kafkaServer.shutdown();
+
+	}
+
+
+
+	// ------------------------------------------------------------------------
+	//  Execution utilities
+	// ------------------------------------------------------------------------
+	
+
+	protected static void tryExecutePropagateExceptions(StreamExecutionEnvironment see, String name) throws Exception {
+		try {
+			see.execute(name);
+		}
+		catch (ProgramInvocationException | JobExecutionException root) {
+			Throwable cause = root.getCause();
+
+			// search for nested SuccessExceptions
+			int depth = 0;
+			while (!(cause instanceof SuccessException)) {
+				if (cause == null || depth++ == 20) {
+					throw root;
+				}
+				else {
+					cause = cause.getCause();
+				}
+			}
+		}
+	}
+
+	protected static void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
+		kafkaServer.createTestTopic(topic, numberOfPartitions, replicationFactor);
+	}
+	
+	protected static void deleteTestTopic(String topic) {
+		kafkaServer.deleteTestTopic(topic);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
new file mode 100644
index 0000000..10c7b86
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import kafka.server.KafkaServer;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+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.streaming.util.serialization.KeyedSerializationSchema;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Properties;
+
+/**
+ * Abstract class providing a Kafka test environment
+ */
+public abstract class KafkaTestEnvironment {
+
+	protected static final String KAFKA_HOST = "localhost";
+
+	public abstract void prepare(int numKafkaServers, Properties kafkaServerProperties, boolean secureMode);
+
+	public void prepare(int numberOfKafkaServers, boolean secureMode) {
+		this.prepare(numberOfKafkaServers, null, secureMode);
+	}
+
+	public abstract void shutdown();
+
+	public abstract void deleteTestTopic(String topic);
+
+	public abstract void createTestTopic(String topic, int numberOfPartitions, int replicationFactor, Properties properties);
+
+	public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
+		this.createTestTopic(topic, numberOfPartitions, replicationFactor, new Properties());
+	}
+
+	public abstract Properties getStandardProperties();
+
+	public abstract Properties getSecureProperties();
+
+	public abstract String getBrokerConnectionString();
+
+	public abstract String getVersion();
+
+	public abstract List<KafkaServer> getBrokers();
+
+	// -- consumer / producer instances:
+	public <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, DeserializationSchema<T> deserializationSchema, Properties props) {
+		return getConsumer(topics, new KeyedDeserializationSchemaWrapper<T>(deserializationSchema), props);
+	}
+
+	public <T> FlinkKafkaConsumerBase<T> getConsumer(String topic, KeyedDeserializationSchema<T> readSchema, Properties props) {
+		return getConsumer(Collections.singletonList(topic), readSchema, props);
+	}
+
+	public <T> FlinkKafkaConsumerBase<T> getConsumer(String topic, DeserializationSchema<T> deserializationSchema, Properties props) {
+		return getConsumer(Collections.singletonList(topic), deserializationSchema, props);
+	}
+
+	public abstract <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, KeyedDeserializationSchema<T> readSchema, Properties props);
+
+	public abstract <T> StreamSink<T> getProducerSink(String topic,
+			KeyedSerializationSchema<T> serSchema, Properties props,
+			KafkaPartitioner<T> partitioner);
+
+	public abstract <T> DataStreamSink<T> produceIntoKafka(DataStream<T> stream, String topic,
+														KeyedSerializationSchema<T> serSchema, Properties props,
+														KafkaPartitioner<T> partitioner);
+
+	// -- offset handlers
+
+	public interface KafkaOffsetHandler {
+		Long getCommittedOffset(String topicName, int partition);
+		void close();
+	}
+
+	public abstract KafkaOffsetHandler createOffsetHandler(Properties props);
+
+	// -- leader failure simulation
+
+	public abstract void restartBroker(int leaderId) throws Exception;
+
+	public abstract int getLeaderToShutDown(String topic) throws Exception;
+
+	public abstract int getBrokerId(KafkaServer server);
+
+	public abstract boolean isSecureRunSupported();
+
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
new file mode 100644
index 0000000..0b3507a
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
@@ -0,0 +1,320 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+@SuppressWarnings("serial")
+public class AbstractFetcherTimestampsTest {
+	
+	@Test
+	public void testPunctuatedWatermarks() throws Exception {
+		final String testTopic = "test topic name";
+		List<KafkaTopicPartition> originalPartitions = Arrays.asList(
+				new KafkaTopicPartition(testTopic, 7),
+				new KafkaTopicPartition(testTopic, 13),
+				new KafkaTopicPartition(testTopic, 21));
+
+		TestSourceContext<Long> sourceContext = new TestSourceContext<>();
+
+		TestProcessingTimeService processingTimeProvider = new TestProcessingTimeService();
+
+		TestFetcher<Long> fetcher = new TestFetcher<>(
+				sourceContext,
+				originalPartitions,
+				null, /* periodic watermark assigner */
+				new SerializedValue<AssignerWithPunctuatedWatermarks<Long>>(new PunctuatedTestExtractor()),
+				processingTimeProvider,
+				0);
+
+		final KafkaTopicPartitionState<Object> part1 = fetcher.subscribedPartitions()[0];
+		final KafkaTopicPartitionState<Object> part2 = fetcher.subscribedPartitions()[1];
+		final KafkaTopicPartitionState<Object> part3 = fetcher.subscribedPartitions()[2];
+
+		// elements generate a watermark if the timestamp is a multiple of three
+		
+		// elements for partition 1
+		fetcher.emitRecord(1L, part1, 1L);
+		fetcher.emitRecord(2L, part1, 2L);
+		fetcher.emitRecord(3L, part1, 3L);
+		assertEquals(3L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(3L, sourceContext.getLatestElement().getTimestamp());
+		assertFalse(sourceContext.hasWatermark());
+
+		// elements for partition 2
+		fetcher.emitRecord(12L, part2, 1L);
+		assertEquals(12L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(12L, sourceContext.getLatestElement().getTimestamp());
+		assertFalse(sourceContext.hasWatermark());
+
+		// elements for partition 3
+		fetcher.emitRecord(101L, part3, 1L);
+		fetcher.emitRecord(102L, part3, 2L);
+		assertEquals(102L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(102L, sourceContext.getLatestElement().getTimestamp());
+		
+		// now, we should have a watermark
+		assertTrue(sourceContext.hasWatermark());
+		assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp());
+		
+		// advance partition 3
+		fetcher.emitRecord(1003L, part3, 3L);
+		fetcher.emitRecord(1004L, part3, 4L);
+		fetcher.emitRecord(1005L, part3, 5L);
+		assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(1005L, sourceContext.getLatestElement().getTimestamp());
+
+		// advance partition 1 beyond partition 2 - this bumps the watermark
+		fetcher.emitRecord(30L, part1, 4L);
+		assertEquals(30L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(30L, sourceContext.getLatestElement().getTimestamp());
+		assertTrue(sourceContext.hasWatermark());
+		assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp());
+
+		// advance partition 2 again - this bumps the watermark
+		fetcher.emitRecord(13L, part2, 2L);
+		assertFalse(sourceContext.hasWatermark());
+		fetcher.emitRecord(14L, part2, 3L);
+		assertFalse(sourceContext.hasWatermark());
+		fetcher.emitRecord(15L, part2, 3L);
+		assertTrue(sourceContext.hasWatermark());
+		assertEquals(15L, sourceContext.getLatestWatermark().getTimestamp());
+	}
+	
+	@Test
+	public void testPeriodicWatermarks() throws Exception {
+		final String testTopic = "test topic name";
+		List<KafkaTopicPartition> originalPartitions = Arrays.asList(
+				new KafkaTopicPartition(testTopic, 7),
+				new KafkaTopicPartition(testTopic, 13),
+				new KafkaTopicPartition(testTopic, 21));
+
+		TestSourceContext<Long> sourceContext = new TestSourceContext<>();
+
+		TestProcessingTimeService processingTimeService = new TestProcessingTimeService();
+
+		TestFetcher<Long> fetcher = new TestFetcher<>(
+				sourceContext,
+				originalPartitions,
+				new SerializedValue<AssignerWithPeriodicWatermarks<Long>>(new PeriodicTestExtractor()),
+				null, /* punctuated watermarks assigner*/
+				processingTimeService,
+				10);
+
+		final KafkaTopicPartitionState<Object> part1 = fetcher.subscribedPartitions()[0];
+		final KafkaTopicPartitionState<Object> part2 = fetcher.subscribedPartitions()[1];
+		final KafkaTopicPartitionState<Object> part3 = fetcher.subscribedPartitions()[2];
+
+		// elements generate a watermark if the timestamp is a multiple of three
+
+		// elements for partition 1
+		fetcher.emitRecord(1L, part1, 1L);
+		fetcher.emitRecord(2L, part1, 2L);
+		fetcher.emitRecord(3L, part1, 3L);
+		assertEquals(3L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(3L, sourceContext.getLatestElement().getTimestamp());
+
+		// elements for partition 2
+		fetcher.emitRecord(12L, part2, 1L);
+		assertEquals(12L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(12L, sourceContext.getLatestElement().getTimestamp());
+
+		// elements for partition 3
+		fetcher.emitRecord(101L, part3, 1L);
+		fetcher.emitRecord(102L, part3, 2L);
+		assertEquals(102L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(102L, sourceContext.getLatestElement().getTimestamp());
+
+		processingTimeService.setCurrentTime(10);
+
+		// now, we should have a watermark (this blocks until the periodic thread emitted the watermark)
+		assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp());
+
+		// advance partition 3
+		fetcher.emitRecord(1003L, part3, 3L);
+		fetcher.emitRecord(1004L, part3, 4L);
+		fetcher.emitRecord(1005L, part3, 5L);
+		assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(1005L, sourceContext.getLatestElement().getTimestamp());
+
+		// advance partition 1 beyond partition 2 - this bumps the watermark
+		fetcher.emitRecord(30L, part1, 4L);
+		assertEquals(30L, sourceContext.getLatestElement().getValue().longValue());
+		assertEquals(30L, sourceContext.getLatestElement().getTimestamp());
+
+		processingTimeService.setCurrentTime(20);
+
+		// this blocks until the periodic thread emitted the watermark
+		assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp());
+
+		// advance partition 2 again - this bumps the watermark
+		fetcher.emitRecord(13L, part2, 2L);
+		fetcher.emitRecord(14L, part2, 3L);
+		fetcher.emitRecord(15L, part2, 3L);
+
+		processingTimeService.setCurrentTime(30);
+		// this blocks until the periodic thread emitted the watermark
+		long watermarkTs = sourceContext.getLatestWatermark().getTimestamp();
+		assertTrue(watermarkTs >= 13L && watermarkTs <= 15L);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Test mocks
+	// ------------------------------------------------------------------------
+
+	private static final class TestFetcher<T> extends AbstractFetcher<T, Object> {
+
+		protected TestFetcher(
+				SourceContext<T> sourceContext,
+				List<KafkaTopicPartition> assignedPartitions,
+				SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+				SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+				ProcessingTimeService processingTimeProvider,
+				long autoWatermarkInterval) throws Exception
+		{
+			super(sourceContext, assignedPartitions, watermarksPeriodic, watermarksPunctuated, processingTimeProvider, autoWatermarkInterval, TestFetcher.class.getClassLoader(), false);
+		}
+
+		@Override
+		public void runFetchLoop() throws Exception {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public void cancel() {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public Object createKafkaPartitionHandle(KafkaTopicPartition partition) {
+			return new Object();
+		}
+
+		@Override
+		public void commitInternalOffsetsToKafka(Map<KafkaTopicPartition, Long> offsets) throws Exception {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static final class TestSourceContext<T> implements SourceContext<T> {
+
+		private final Object checkpointLock = new Object();
+		private final Object watermarkLock = new Object();
+
+		private volatile StreamRecord<T> latestElement;
+		private volatile Watermark currentWatermark;
+
+		@Override
+		public void collect(T element) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public void collectWithTimestamp(T element, long timestamp) {
+			this.latestElement = new StreamRecord<>(element, timestamp);
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+			synchronized (watermarkLock) {
+				currentWatermark = mark;
+				watermarkLock.notifyAll();
+			}
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return checkpointLock;
+		}
+
+		@Override
+		public void close() {}
+
+		public StreamRecord<T> getLatestElement() {
+			return latestElement;
+		}
+
+		public boolean hasWatermark() {
+			return currentWatermark != null;
+		}
+		
+		public Watermark getLatestWatermark() throws InterruptedException {
+			synchronized (watermarkLock) {
+				while (currentWatermark == null) {
+					watermarkLock.wait();
+				}
+				Watermark wm = currentWatermark;
+				currentWatermark = null;
+				return wm;
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static class PeriodicTestExtractor implements AssignerWithPeriodicWatermarks<Long> {
+
+		private volatile long maxTimestamp = Long.MIN_VALUE;
+		
+		@Override
+		public long extractTimestamp(Long element, long previousElementTimestamp) {
+			maxTimestamp = Math.max(maxTimestamp, element);
+			return element;
+		}
+
+		@Nullable
+		@Override
+		public Watermark getCurrentWatermark() {
+			return new Watermark(maxTimestamp);
+		}
+	}
+
+	private static class PunctuatedTestExtractor implements AssignerWithPunctuatedWatermarks<Long> {
+
+		@Override
+		public long extractTimestamp(Long element, long previousElementTimestamp) {
+			return element;
+		}
+
+		@Nullable
+		@Override
+		public Watermark checkAndGetNextWatermark(Long lastElement, long extractedTimestamp) {
+			return extractedTimestamp % 3 == 0 ? new Watermark(extractedTimestamp) : null;
+		}
+		
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java b/flink-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-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/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
new file mode 100644
index 0000000..9e8e1d9
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase;
+import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironment;
+import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
+
+import java.util.Collection;
+import java.util.Properties;
+import java.util.Random;
+
+import static org.mockito.Mockito.mock;
+
+@SuppressWarnings("serial")
+public class DataGenerators {
+
+	public static void generateRandomizedIntegerSequence(StreamExecutionEnvironment env,
+														 KafkaTestEnvironment testServer, String topic,
+														 final int numPartitions,
+														 final int numElements,
+														 final boolean randomizeOrder) throws Exception {
+		env.setParallelism(numPartitions);
+		env.getConfig().disableSysoutLogging();
+		env.setRestartStrategy(RestartStrategies.noRestart());
+
+		DataStream<Integer> stream = env.addSource(
+				new RichParallelSourceFunction<Integer>() {
+
+					private volatile boolean running = true;
+
+					@Override
+					public void run(SourceContext<Integer> ctx) {
+						// create a sequence
+						int[] elements = new int[numElements];
+						for (int i = 0, val = getRuntimeContext().getIndexOfThisSubtask();
+							 i < numElements;
+							 i++, val += getRuntimeContext().getNumberOfParallelSubtasks()) {
+
+							elements[i] = val;
+						}
+
+						// scramble the sequence
+						if (randomizeOrder) {
+							Random rnd = new Random();
+							for (int i = 0; i < elements.length; i++) {
+								int otherPos = rnd.nextInt(elements.length);
+
+								int tmp = elements[i];
+								elements[i] = elements[otherPos];
+								elements[otherPos] = tmp;
+							}
+						}
+
+						// emit the sequence
+						int pos = 0;
+						while (running && pos < elements.length) {
+							ctx.collect(elements[pos++]);
+						}
+					}
+
+					@Override
+					public void cancel() {
+						running = false;
+					}
+				});
+
+		Properties props = new Properties();
+		props.putAll(FlinkKafkaProducerBase.getPropertiesFromBrokerList(testServer.getBrokerConnectionString()));
+		Properties secureProps = testServer.getSecureProperties();
+		if(secureProps != null) {
+			props.putAll(testServer.getSecureProperties());
+		}
+
+		stream = stream.rebalance();
+		testServer.produceIntoKafka(stream, topic,
+				new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, env.getConfig())),
+				props,
+				new KafkaPartitioner<Integer>() {
+					@Override
+					public int partition(Integer next, byte[] serializedKey, byte[] serializedValue, int numPartitions) {
+						return next % numPartitions;
+					}
+				});
+
+		env.execute("Scrambles int sequence generator");
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static class InfiniteStringsGenerator extends Thread {
+
+		private final KafkaTestEnvironment server;
+
+		private final String topic;
+
+		private volatile Throwable error;
+
+		private volatile boolean running = true;
+
+
+		public InfiniteStringsGenerator(KafkaTestEnvironment server, String topic) {
+			this.server = server;
+			this.topic = topic;
+		}
+
+		@Override
+		public void run() {
+			// we manually feed data into the Kafka sink
+			RichFunction producer = null;
+			try {
+				Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(server.getBrokerConnectionString());
+				producerProperties.setProperty("retries", "3");
+				StreamTransformation<String> mockTransform = new MockStreamTransformation();
+				DataStream<String> stream = new DataStream<>(new DummyStreamExecutionEnvironment(), mockTransform);
+
+				StreamSink<String> sink = server.getProducerSink(
+						topic,
+						new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()),
+						producerProperties,
+						new FixedPartitioner<String>());
+
+				OneInputStreamOperatorTestHarness<String, Object> testHarness =
+						new OneInputStreamOperatorTestHarness<>(sink);
+
+				testHarness.open();
+
+				final StringBuilder bld = new StringBuilder();
+				final Random rnd = new Random();
+
+				while (running) {
+					bld.setLength(0);
+
+					int len = rnd.nextInt(100) + 1;
+					for (int i = 0; i < len; i++) {
+						bld.append((char) (rnd.nextInt(20) + 'a') );
+					}
+
+					String next = bld.toString();
+					testHarness.processElement(new StreamRecord<>(next));
+				}
+			}
+			catch (Throwable t) {
+				this.error = t;
+			}
+			finally {
+				if (producer != null) {
+					try {
+						producer.close();
+					}
+					catch (Throwable t) {
+						// ignore
+					}
+				}
+			}
+		}
+
+		public void shutdown() {
+			this.running = false;
+			this.interrupt();
+		}
+
+		public Throwable getError() {
+			return this.error;
+		}
+
+		private static class MockStreamTransformation extends StreamTransformation<String> {
+			public MockStreamTransformation() {
+				super("MockTransform", TypeInfoParser.<String>parse("String"), 1);
+			}
+
+			@Override
+			public void setChainingStrategy(ChainingStrategy strategy) {
+
+			}
+
+			@Override
+			public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+				return null;
+			}
+		}
+
+		public static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment {
+
+			@Override
+			public JobExecutionResult execute(String jobName) throws Exception {
+				return null;
+			}
+		}
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java
new file mode 100644
index 0000000..055326d
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+
+import java.util.Properties;
+
+public class FakeStandardProducerConfig {
+
+	public static Properties get() {
+		Properties p = new Properties();
+		p.setProperty("bootstrap.servers", "localhost:12345");
+		p.setProperty("key.serializer", ByteArraySerializer.class.getName());
+		p.setProperty("value.serializer", ByteArraySerializer.class.getName());
+		return p;
+	}
+
+}

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

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

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

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

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

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

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

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


[10/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index c28799c..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.junit.Test;
-
-@SuppressWarnings("serial")
-public class KafkaShortRetention08ITCase extends KafkaShortRetentionTestBase {
-
-	@Test(timeout=60000)
-	public void testAutoOffsetReset() throws Exception {
-		runAutoOffsetResetTest();
-	}
-
-	@Test(timeout=60000)
-	public void testAutoOffsetResetNone() throws Exception {
-		runFailOnAutoOffsetResetNoneEager();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
deleted file mode 100644
index 6235449..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
+++ /dev/null
@@ -1,401 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import kafka.admin.AdminUtils;
-import kafka.api.PartitionMetadata;
-import kafka.common.KafkaException;
-import kafka.network.SocketServer;
-import kafka.server.KafkaConfig;
-import kafka.server.KafkaServer;
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.commons.io.FileUtils;
-import org.apache.curator.RetryPolicy;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.curator.test.TestingServer;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader;
-import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
-import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.util.NetUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.collection.Seq;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.BindException;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-import java.util.UUID;
-
-import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * An implementation of the KafkaServerProvider for Kafka 0.8
- */
-public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment {
-
-	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class);
-	private File tmpZkDir;
-	private File tmpKafkaParent;
-	private List<File> tmpKafkaDirs;
-	private List<KafkaServer> brokers;
-	private TestingServer zookeeper;
-	private String zookeeperConnectionString;
-	private String brokerConnectionString = "";
-	private Properties standardProps;
-	private Properties additionalServerProperties;
-
-	public String getBrokerConnectionString() {
-		return brokerConnectionString;
-	}
-
-	@Override
-	public Properties getStandardProperties() {
-		return standardProps;
-	}
-
-	@Override
-	public Properties getSecureProperties() {
-		return null;
-	}
-
-	@Override
-	public String getVersion() {
-		return "0.8";
-	}
-
-	@Override
-	public List<KafkaServer> getBrokers() {
-		return brokers;
-	}
-
-	@Override
-	public <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, KeyedDeserializationSchema<T> readSchema, Properties props) {
-		return new FlinkKafkaConsumer08<>(topics, readSchema, props);
-	}
-
-	@Override
-	public <T> StreamSink<T> getProducerSink(
-			String topic,
-			KeyedSerializationSchema<T> serSchema,
-			Properties props,
-			KafkaPartitioner<T> partitioner) {
-		FlinkKafkaProducer08<T> prod = new FlinkKafkaProducer08<>(
-				topic,
-				serSchema,
-				props,
-				partitioner);
-		prod.setFlushOnCheckpoint(true);
-		return new StreamSink<>(prod);
-	}
-
-	@Override
-	public <T> DataStreamSink<T> produceIntoKafka(DataStream<T> stream, String topic, KeyedSerializationSchema<T> serSchema, Properties props, KafkaPartitioner<T> partitioner) {
-		FlinkKafkaProducer08<T> prod = new FlinkKafkaProducer08<>(topic, serSchema, props, partitioner);
-		prod.setFlushOnCheckpoint(true);
-		return stream.addSink(prod);
-	}
-
-	@Override
-	public KafkaOffsetHandler createOffsetHandler(Properties props) {
-		return new KafkaOffsetHandlerImpl(props);
-	}
-
-	@Override
-	public void restartBroker(int leaderId) throws Exception {
-		brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId)));
-	}
-
-	@Override
-	public int getLeaderToShutDown(String topic) throws Exception {
-		ZkClient zkClient = createZkClient();
-		PartitionMetadata firstPart = null;
-		do {
-			if (firstPart != null) {
-				LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
-				// not the first try. Sleep a bit
-				Thread.sleep(150);
-			}
-
-			Seq<PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata();
-			firstPart = partitionMetadata.head();
-		}
-		while (firstPart.errorCode() != 0);
-		zkClient.close();
-
-		return firstPart.leader().get().id();
-	}
-
-	@Override
-	public int getBrokerId(KafkaServer server) {
-		return server.socketServer().brokerId();
-	}
-
-	@Override
-	public boolean isSecureRunSupported() {
-		return false;
-	}
-
-
-	@Override
-	public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) {
-		this.additionalServerProperties = additionalServerProperties;
-		File tempDir = new File(System.getProperty("java.io.tmpdir"));
-
-		tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
-		try {
-			Files.createDirectories(tmpZkDir.toPath());
-		} catch (IOException e) {
-			fail("cannot create zookeeper temp dir: " + e.getMessage());
-		}
-
-		tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir" + (UUID.randomUUID().toString()));
-		try {
-			Files.createDirectories(tmpKafkaParent.toPath());
-		} catch (IOException e) {
-			fail("cannot create kafka temp dir: " + e.getMessage());
-		}
-
-		tmpKafkaDirs = new ArrayList<>(numKafkaServers);
-		for (int i = 0; i < numKafkaServers; i++) {
-			File tmpDir = new File(tmpKafkaParent, "server-" + i);
-			assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
-			tmpKafkaDirs.add(tmpDir);
-		}
-
-		zookeeper = null;
-		brokers = null;
-
-		try {
-			LOG.info("Starting Zookeeper");
-			zookeeper = new TestingServer(-1, tmpZkDir);
-			zookeeperConnectionString = zookeeper.getConnectString();
-
-			LOG.info("Starting KafkaServer");
-			brokers = new ArrayList<>(numKafkaServers);
-
-			for (int i = 0; i < numKafkaServers; i++) {
-				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i)));
-				SocketServer socketServer = brokers.get(i).socketServer();
-
-				String host = socketServer.host() == null ? "localhost" : socketServer.host();
-				brokerConnectionString += hostAndPortToUrlString(host, socketServer.port()) + ",";
-			}
-
-			LOG.info("ZK and KafkaServer started.");
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("Test setup failed: " + t.getMessage());
-		}
-
-		standardProps = new Properties();
-		standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
-		standardProps.setProperty("bootstrap.servers", brokerConnectionString);
-		standardProps.setProperty("group.id", "flink-tests");
-		standardProps.setProperty("auto.commit.enable", "false");
-		standardProps.setProperty("zookeeper.session.timeout.ms", "30000"); // 6 seconds is default. Seems to be too small for travis.
-		standardProps.setProperty("zookeeper.connection.timeout.ms", "30000");
-		standardProps.setProperty("auto.offset.reset", "smallest"); // read from the beginning. (smallest is kafka 0.8)
-		standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
-	}
-
-	@Override
-	public void shutdown() {
-		if (brokers != null) {
-			for (KafkaServer broker : brokers) {
-				if (broker != null) {
-					broker.shutdown();
-				}
-			}
-			brokers.clear();
-		}
-
-		if (zookeeper != null) {
-			try {
-				zookeeper.stop();
-				zookeeper.close();
-			}
-			catch (Exception e) {
-				LOG.warn("ZK.stop() failed", e);
-			}
-			zookeeper = null;
-		}
-
-		// clean up the temp spaces
-
-		if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
-			try {
-				FileUtils.deleteDirectory(tmpKafkaParent);
-			}
-			catch (Exception e) {
-				// ignore
-			}
-		}
-		if (tmpZkDir != null && tmpZkDir.exists()) {
-			try {
-				FileUtils.deleteDirectory(tmpZkDir);
-			}
-			catch (Exception e) {
-				// ignore
-			}
-		}
-	}
-
-	@Override
-	public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor, Properties topicConfig) {
-		// create topic with one client
-		LOG.info("Creating topic {}", topic);
-
-		ZkClient creator = createZkClient();
-
-		AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig);
-		creator.close();
-
-		// validate that the topic has been created
-		final long deadline = System.currentTimeMillis() + 30000;
-		do {
-			try {
-				Thread.sleep(100);
-			}
-			catch (InterruptedException e) {
-				// restore interrupted state
-			}
-			List<KafkaTopicPartitionLeader> partitions = FlinkKafkaConsumer08.getPartitionsForTopic(Collections.singletonList(topic), standardProps);
-			if (partitions != null && partitions.size() > 0) {
-				return;
-			}
-		}
-		while (System.currentTimeMillis() < deadline);
-		fail ("Test topic could not be created");
-	}
-
-	@Override
-	public void deleteTestTopic(String topic) {
-		LOG.info("Deleting topic {}", topic);
-
-		ZkClient zk = createZkClient();
-		AdminUtils.deleteTopic(zk, topic);
-		zk.close();
-	}
-
-	private ZkClient createZkClient() {
-		return new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
-				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
-	}
-
-	/**
-	 * Only for the 0.8 server we need access to the zk client.
-	 */
-	public CuratorFramework createCuratorClient() {
-		RetryPolicy retryPolicy = new ExponentialBackoffRetry(100, 10);
-		CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(standardProps.getProperty("zookeeper.connect"), retryPolicy);
-		curatorClient.start();
-		return curatorClient;
-	}
-
-	/**
-	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
-	 */
-	protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception {
-		LOG.info("Starting broker with id {}", brokerId);
-		Properties kafkaProperties = new Properties();
-
-		// properties have to be Strings
-		kafkaProperties.put("advertised.host.name", KAFKA_HOST);
-		kafkaProperties.put("broker.id", Integer.toString(brokerId));
-		kafkaProperties.put("log.dir", tmpFolder.toString());
-		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
-		kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
-		kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
-
-		// for CI stability, increase zookeeper session timeout
-		kafkaProperties.put("zookeeper.session.timeout.ms", "30000");
-		kafkaProperties.put("zookeeper.connection.timeout.ms", "30000");
-		if(additionalServerProperties != null) {
-			kafkaProperties.putAll(additionalServerProperties);
-		}
-
-		final int numTries = 5;
-
-		for (int i = 1; i <= numTries; i++) {
-			int kafkaPort = NetUtils.getAvailablePort();
-			kafkaProperties.put("port", Integer.toString(kafkaPort));
-			KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
-
-			try {
-				KafkaServer server = new KafkaServer(kafkaConfig, new KafkaLocalSystemTime());
-				server.startup();
-				return server;
-			}
-			catch (KafkaException e) {
-				if (e.getCause() instanceof BindException) {
-					// port conflict, retry...
-					LOG.info("Port conflict when starting Kafka Broker. Retrying...");
-				}
-				else {
-					throw e;
-				}
-			}
-		}
-
-		throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
-	}
-
-	private class KafkaOffsetHandlerImpl implements KafkaOffsetHandler {
-
-		private final CuratorFramework offsetClient;
-		private final String groupId;
-
-		public KafkaOffsetHandlerImpl(Properties props) {
-			offsetClient = createCuratorClient();
-			groupId = props.getProperty("group.id");
-		}
-
-		@Override
-		public Long getCommittedOffset(String topicName, int partition) {
-			try {
-				return ZookeeperOffsetHandler.getOffsetFromZooKeeper(offsetClient, groupId, topicName, partition);
-			} catch (Exception e) {
-				throw new RuntimeException("Exception when getting offsets from Zookeeper", e);
-			}
-		}
-
-		@Override
-		public void close() {
-			offsetClient.close();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java
deleted file mode 100644
index 6298c92..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java
+++ /dev/null
@@ -1,603 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.*;
-import static java.util.Arrays.asList;
-import static java.util.Collections.emptyList;
-import static java.util.Collections.singletonList;
-
-public class ClosableBlockingQueueTest {
-
-	// ------------------------------------------------------------------------
-	//  single-threaded unit tests
-	// ------------------------------------------------------------------------
-	
-	@Test
-	public void testCreateQueueHashCodeEquals() {
-		try {
-			ClosableBlockingQueue<String> queue1 = new ClosableBlockingQueue<>();
-			ClosableBlockingQueue<String> queue2 = new ClosableBlockingQueue<>(22);
-
-			assertTrue(queue1.isOpen());
-			assertTrue(queue2.isOpen());
-			assertTrue(queue1.isEmpty());
-			assertTrue(queue2.isEmpty());
-			assertEquals(0, queue1.size());
-			assertEquals(0, queue2.size());
-			
-			assertTrue(queue1.hashCode() == queue2.hashCode());
-			//noinspection EqualsWithItself
-			assertTrue(queue1.equals(queue1));
-			//noinspection EqualsWithItself
-			assertTrue(queue2.equals(queue2));
-			assertTrue(queue1.equals(queue2));
-			
-			assertNotNull(queue1.toString());
-			assertNotNull(queue2.toString());
-
-			List<String> elements = new ArrayList<>();
-			elements.add("a");
-			elements.add("b");
-			elements.add("c");
-
-			ClosableBlockingQueue<String> queue3 = new ClosableBlockingQueue<>(elements);
-			ClosableBlockingQueue<String> queue4 = new ClosableBlockingQueue<>(asList("a", "b", "c"));
-
-			assertTrue(queue3.isOpen());
-			assertTrue(queue4.isOpen());
-			assertFalse(queue3.isEmpty());
-			assertFalse(queue4.isEmpty());
-			assertEquals(3, queue3.size());
-			assertEquals(3, queue4.size());
-
-			assertTrue(queue3.hashCode() == queue4.hashCode());
-			//noinspection EqualsWithItself
-			assertTrue(queue3.equals(queue3));
-			//noinspection EqualsWithItself
-			assertTrue(queue4.equals(queue4));
-			assertTrue(queue3.equals(queue4));
-			
-			assertNotNull(queue3.toString());
-			assertNotNull(queue4.toString());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testCloseEmptyQueue() {
-		try {
-			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
-			assertTrue(queue.isOpen());
-			assertTrue(queue.close());
-			assertFalse(queue.isOpen());
-			
-			assertFalse(queue.addIfOpen("element"));
-			assertTrue(queue.isEmpty());
-			
-			try {
-				queue.add("some element");
-				fail("should cause an exception");
-			} catch (IllegalStateException ignored) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testCloseNonEmptyQueue() {
-		try {
-			ClosableBlockingQueue<Integer> queue = new ClosableBlockingQueue<>(asList(1, 2, 3));
-			assertTrue(queue.isOpen());
-			
-			assertFalse(queue.close());
-			assertFalse(queue.close());
-			
-			queue.poll();
-
-			assertFalse(queue.close());
-			assertFalse(queue.close());
-			
-			queue.pollBatch();
-
-			assertTrue(queue.close());
-			assertFalse(queue.isOpen());
-
-			assertFalse(queue.addIfOpen(42));
-			assertTrue(queue.isEmpty());
-
-			try {
-				queue.add(99);
-				fail("should cause an exception");
-			} catch (IllegalStateException ignored) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testPeekAndPoll() {
-		try {
-			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
-			
-			assertNull(queue.peek());
-			assertNull(queue.peek());
-			assertNull(queue.poll());
-			assertNull(queue.poll());
-			
-			assertEquals(0, queue.size());
-			
-			queue.add("a");
-			queue.add("b");
-			queue.add("c");
-
-			assertEquals(3, queue.size());
-			
-			assertEquals("a", queue.peek());
-			assertEquals("a", queue.peek());
-			assertEquals("a", queue.peek());
-
-			assertEquals(3, queue.size());
-			
-			assertEquals("a", queue.poll());
-			assertEquals("b", queue.poll());
-
-			assertEquals(1, queue.size());
-			
-			assertEquals("c", queue.peek());
-			assertEquals("c", queue.peek());
-
-			assertEquals("c", queue.poll());
-
-			assertEquals(0, queue.size());
-			assertNull(queue.poll());
-			assertNull(queue.peek());
-			assertNull(queue.peek());
-			
-			assertTrue(queue.close());
-			
-			try {
-				queue.peek();
-				fail("should cause an exception");
-			} catch (IllegalStateException ignored) {
-				// expected
-			}
-
-			try {
-				queue.poll();
-				fail("should cause an exception");
-			} catch (IllegalStateException ignored) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testPollBatch() {
-		try {
-			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
-
-			assertNull(queue.pollBatch());
-			
-			queue.add("a");
-			queue.add("b");
-			
-			assertEquals(asList("a", "b"), queue.pollBatch());
-			assertNull(queue.pollBatch());
-			
-			queue.add("c");
-
-			assertEquals(singletonList("c"), queue.pollBatch());
-			assertNull(queue.pollBatch());
-
-			assertTrue(queue.close());
-
-			try {
-				queue.pollBatch();
-				fail("should cause an exception");
-			} catch (IllegalStateException ignored) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testGetElementBlocking() {
-		try {
-			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
-
-			assertNull(queue.getElementBlocking(1));
-			assertNull(queue.getElementBlocking(3));
-			assertNull(queue.getElementBlocking(2));
-
-			assertEquals(0, queue.size());
-
-			queue.add("a");
-			queue.add("b");
-			queue.add("c");
-			queue.add("d");
-			queue.add("e");
-			queue.add("f");
-
-			assertEquals(6, queue.size());
-
-			assertEquals("a", queue.getElementBlocking(99));
-			assertEquals("b", queue.getElementBlocking());
-
-			assertEquals(4, queue.size());
-
-			assertEquals("c", queue.getElementBlocking(0));
-			assertEquals("d", queue.getElementBlocking(1000000));
-			assertEquals("e", queue.getElementBlocking());
-			assertEquals("f", queue.getElementBlocking(1786598));
-
-			assertEquals(0, queue.size());
-
-			assertNull(queue.getElementBlocking(1));
-			assertNull(queue.getElementBlocking(3));
-			assertNull(queue.getElementBlocking(2));
-
-			assertTrue(queue.close());
-
-			try {
-				queue.getElementBlocking();
-				fail("should cause an exception");
-			} catch (IllegalStateException ignored) {
-				// expected
-			}
-
-			try {
-				queue.getElementBlocking(1000000000L);
-				fail("should cause an exception");
-			} catch (IllegalStateException ignored) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testGetBatchBlocking() {
-		try {
-			ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
-
-			assertEquals(emptyList(), queue.getBatchBlocking(1));
-			assertEquals(emptyList(), queue.getBatchBlocking(3));
-			assertEquals(emptyList(), queue.getBatchBlocking(2));
-
-			queue.add("a");
-			queue.add("b");
-
-			assertEquals(asList("a", "b"), queue.getBatchBlocking(900000009));
-
-			queue.add("c");
-			queue.add("d");
-
-			assertEquals(asList("c", "d"), queue.getBatchBlocking());
-
-			assertEquals(emptyList(), queue.getBatchBlocking(2));
-
-			queue.add("e");
-
-			assertEquals(singletonList("e"), queue.getBatchBlocking(0));
-
-			queue.add("f");
-
-			assertEquals(singletonList("f"), queue.getBatchBlocking(1000000000));
-
-			assertEquals(0, queue.size());
-
-			assertEquals(emptyList(), queue.getBatchBlocking(1));
-			assertEquals(emptyList(), queue.getBatchBlocking(3));
-			assertEquals(emptyList(), queue.getBatchBlocking(2));
-
-			assertTrue(queue.close());
-
-			try {
-				queue.getBatchBlocking();
-				fail("should cause an exception");
-			} catch (IllegalStateException ignored) {
-				// expected
-			}
-
-			try {
-				queue.getBatchBlocking(1000000000L);
-				fail("should cause an exception");
-			} catch (IllegalStateException ignored) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  multi-threaded tests
-	// ------------------------------------------------------------------------
-	
-	@Test
-	public void notifyOnClose() {
-		try {
-			final long oneYear = 365L * 24 * 60 * 60 * 1000;
-			
-			// test "getBatchBlocking()"
-			final ClosableBlockingQueue<String> queue1 = new ClosableBlockingQueue<>();
-			QueueCall call1 = new QueueCall() {
-				@Override
-				public void call() throws Exception {
-					queue1.getBatchBlocking();
-				}
-			};
-			testCallExitsOnClose(call1, queue1);
-
-			// test "getBatchBlocking()"
-			final ClosableBlockingQueue<String> queue2 = new ClosableBlockingQueue<>();
-			QueueCall call2 = new QueueCall() {
-				@Override
-				public void call() throws Exception {
-					queue2.getBatchBlocking(oneYear);
-				}
-			};
-			testCallExitsOnClose(call2, queue2);
-
-			// test "getBatchBlocking()"
-			final ClosableBlockingQueue<String> queue3 = new ClosableBlockingQueue<>();
-			QueueCall call3 = new QueueCall() {
-				@Override
-				public void call() throws Exception {
-					queue3.getElementBlocking();
-				}
-			};
-			testCallExitsOnClose(call3, queue3);
-
-			// test "getBatchBlocking()"
-			final ClosableBlockingQueue<String> queue4 = new ClosableBlockingQueue<>();
-			QueueCall call4 = new QueueCall() {
-				@Override
-				public void call() throws Exception {
-					queue4.getElementBlocking(oneYear);
-				}
-			};
-			testCallExitsOnClose(call4, queue4);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-	@Test
-	public void testMultiThreadedAddGet() {
-		try {
-			final ClosableBlockingQueue<Integer> queue = new ClosableBlockingQueue<>();
-			final AtomicReference<Throwable> pushErrorRef = new AtomicReference<>();
-			final AtomicReference<Throwable> pollErrorRef = new AtomicReference<>();
-			
-			final int numElements = 2000;
-			
-			Thread pusher = new Thread("pusher") {
-
-				@Override
-				public void run() {
-					try {
-						final Random rnd = new Random();
-						for (int i = 0; i < numElements; i++) {
-							queue.add(i);
-							
-							// sleep a bit, sometimes
-							int sleepTime = rnd.nextInt(3);
-							if (sleepTime > 1) {
-								Thread.sleep(sleepTime);
-							}
-						}
-						
-						while (true) {
-							if (queue.close()) {
-								break;
-							} else {
-								Thread.sleep(5);
-							}
-						}
-					} catch (Throwable t) {
-						pushErrorRef.set(t);
-					}
-				}
-			};
-			pusher.start();
-
-			Thread poller = new Thread("poller") {
-
-				@SuppressWarnings("InfiniteLoopStatement")
-				@Override
-				public void run() {
-					try {
-						int count = 0;
-						
-						try {
-							final Random rnd = new Random();
-							int nextExpected = 0;
-							
-							while (true) {
-								int getMethod = count % 7;
-								switch (getMethod) {
-									case 0: {
-										Integer next = queue.getElementBlocking(1);
-										if (next != null) {
-											assertEquals(nextExpected, next.intValue());
-											nextExpected++;
-											count++;
-										}
-										break;
-									}
-									case 1: {
-										List<Integer> nextList = queue.getBatchBlocking();
-										for (Integer next : nextList) {
-											assertNotNull(next);
-											assertEquals(nextExpected, next.intValue());
-											nextExpected++;
-											count++;
-										}
-										break;
-									}
-									case 2: {
-										List<Integer> nextList = queue.getBatchBlocking(1);
-										if (nextList != null) {
-											for (Integer next : nextList) {
-												assertNotNull(next);
-												assertEquals(nextExpected, next.intValue());
-												nextExpected++;
-												count++;
-											}
-										}
-										break;
-									}
-									case 3: {
-										Integer next = queue.poll();
-										if (next != null) {
-											assertEquals(nextExpected, next.intValue());
-											nextExpected++;
-											count++;
-										}
-										break;
-									}
-									case 4: {
-										List<Integer> nextList = queue.pollBatch();
-										if (nextList != null) {
-											for (Integer next : nextList) {
-												assertNotNull(next);
-												assertEquals(nextExpected, next.intValue());
-												nextExpected++;
-												count++;
-											}
-										}
-										break;
-									}
-									default: {
-										Integer next = queue.getElementBlocking();
-										assertNotNull(next);
-										assertEquals(nextExpected, next.intValue());
-										nextExpected++;
-										count++;
-									}
-								}
-								
-								// sleep a bit, sometimes
-								int sleepTime = rnd.nextInt(3);
-								if (sleepTime > 1) {
-									Thread.sleep(sleepTime);
-								}
-							}
-						} catch (IllegalStateException e) {
-							// we get this once the queue is closed
-							assertEquals(numElements, count);
-						}
-					} catch (Throwable t) {
-						pollErrorRef.set(t);
-					}
-				}
-			};
-			poller.start();
-			
-			pusher.join();
-			poller.join();
-			
-			if (pushErrorRef.get() != null) {
-				Throwable t = pushErrorRef.get();
-				t.printStackTrace();
-				fail("Error in pusher: " + t.getMessage());
-			}
-			if (pollErrorRef.get() != null) {
-				Throwable t = pollErrorRef.get();
-				t.printStackTrace();
-				fail("Error in poller: " + t.getMessage());
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utils
-	// ------------------------------------------------------------------------
-	
-	private static void testCallExitsOnClose(
-			final QueueCall call, ClosableBlockingQueue<String> queue) throws Exception {
-		
-		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
-		
-		Runnable runnable = new Runnable() {
-			@Override
-			public void run() {
-				try {
-					call.call();
-				} catch (Throwable t) {
-					errorRef.set(t);
-				}
-			}
-		};
-
-		Thread thread = new Thread(runnable);
-		thread.start();
-		Thread.sleep(100);
-		queue.close();
-		thread.join();
-
-		@SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-		Throwable cause = errorRef.get();
-		assertTrue(cause instanceof IllegalStateException);
-	}
-	
-	private interface QueueCall {
-		void call() throws Exception;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index fbeb110..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,30 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-log4j.rootLogger=INFO, testlogger
-
-log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
-log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
-log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
-
-# suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
-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/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/logback-test.xml
deleted file mode 100644
index 45b3b92..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,30 +0,0 @@
-<!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one
-  ~ or more contributor license agreements.  See the NOTICE file
-  ~ distributed with this work for additional information
-  ~ regarding copyright ownership.  The ASF licenses this file
-  ~ to you under the Apache License, Version 2.0 (the
-  ~ "License"); you may not use this file except in compliance
-  ~ with the License.  You may obtain a copy of the License at
-  ~
-  ~     http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
-
-<configuration>
-    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <root level="WARN">
-        <appender-ref ref="STDOUT"/>
-    </root>
-    <logger name="org.apache.flink.streaming" level="WARN"/>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml
deleted file mode 100644
index f638c7a..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml
+++ /dev/null
@@ -1,212 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-kafka-0.9_2.10</artifactId>
-	<name>flink-connector-kafka-0.9</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<kafka.version>0.9.0.1</kafka.version>
-	</properties>
-
-	<dependencies>
-
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
-			<version>${project.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.apache.kafka</groupId>
-					<artifactId>kafka_${scala.binary.version}</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-			<!-- Projects depending on this project,
-			won't depend on flink-table. -->
-			<optional>true</optional>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.kafka</groupId>
-			<artifactId>kafka-clients</artifactId>
-			<version>${kafka.version}</version>
-		</dependency>
-
-		<!-- test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
-			<version>${project.version}</version>
-			<exclusions>
-				<!-- exclude 0.8 dependencies -->
-				<exclusion>
-					<groupId>org.apache.kafka</groupId>
-					<artifactId>kafka_${scala.binary.version}</artifactId>
-				</exclusion>
-			</exclusions>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<!-- include 0.9 server for tests  -->
-			<groupId>org.apache.kafka</groupId>
-			<artifactId>kafka_${scala.binary.version}</artifactId>
-			<version>${kafka.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-metrics-jmx</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-		
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-minikdc</artifactId>
-			<version>${minikdc.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-						<configuration>
-							<includes>
-								<include>**/KafkaTestEnvironmentImpl*</include>
-							</includes>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-source-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>attach-test-sources</id>
-						<goals>
-							<goal>test-jar-no-fork</goal>
-						</goals>
-						<configuration>
-							<includes>
-								<include>**/KafkaTestEnvironmentImpl*</include>
-							</includes>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
-					<forkCount>1</forkCount>
-					<argLine>-Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
-				</configuration>
-			</plugin>
-			<!--
-            https://issues.apache.org/jira/browse/DIRSHARED-134
-            Required to pull the Mini-KDC transitive dependency
-            -->
-			<plugin>
-				<groupId>org.apache.felix</groupId>
-				<artifactId>maven-bundle-plugin</artifactId>
-				<version>3.0.1</version>
-				<inherited>true</inherited>
-				<extensions>true</extensions>
-			</plugin>
-		</plugins>
-	</build>
-	
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index 29bb8e4..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
+++ /dev/null
@@ -1,269 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-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.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.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.KafkaConsumer;
-import org.apache.kafka.common.PartitionInfo;
-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.List;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
- * Apache Kafka 0.9.x. The consumer can run in multiple parallel instances, each of which will pull
- * data from one or more Kafka partitions. 
- * 
- * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
- * during a failure, and that the computation processes elements "exactly once". 
- * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
- *
- * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
- * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
- * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
- * has consumed a topic.</p>
- *
- * <p>Please refer to Kafka's documentation for the available configuration properties:
- * http://kafka.apache.org/documentation.html#newconsumerconfigs</p>
- *
- * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
- * is constructed. That means that the client that submits the program needs to be able to
- * reach the Kafka brokers or ZooKeeper.</p>
- */
-public class 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 **/
-	public static final String KEY_POLL_TIMEOUT = "flink.poll-timeout";
-
-
-	/** 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 **/
-	protected final Properties properties;
-
-	/** 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 */
-	protected final long pollTimeout;
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.9.x
-	 *
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param valueDeserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer09(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		this(Collections.singletonList(topic), valueDeserializer, props);
-	}
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.9.x
-	 *
-	 * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value
-	 * pairs, offsets, and topic names from Kafka.
-	 *
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param deserializer
-	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer09(String topic, KeyedDeserializationSchema<T> deserializer, Properties props) {
-		this(Collections.singletonList(topic), deserializer, props);
-	}
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.9.x
-	 *
-	 * This constructor allows passing multiple topics to the consumer.
-	 *
-	 * @param topics
-	 *           The Kafka topics to read from.
-	 * @param deserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           The properties that are used to configure both the fetcher and the offset handler.
-	 */
-	public FlinkKafkaConsumer09(List<String> topics, DeserializationSchema<T> deserializer, Properties props) {
-		this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props);
-	}
-
-	/**
-	 * Creates a new Kafka streaming source consumer for Kafka 0.9.x
-	 *
-	 * This constructor allows passing multiple topics and a key/value deserialization schema.
-	 *
-	 * @param topics
-	 *           The Kafka topics to read from.
-	 * @param deserializer
-	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param props
-	 *           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(topics, deserializer);
-
-		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);
-		}
-	}
-
-	@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.getProcessingTimeService(),
-				runtimeContext.getExecutionConfig().getAutoWatermarkInterval(),
-				runtimeContext.getUserCodeClassLoader(),
-				runtimeContext.isCheckpointingEnabled(),
-				runtimeContext.getTaskNameWithSubtasks(),
-				runtimeContext.getMetricGroup(),
-				deserializer,
-				properties,
-				pollTimeout,
-				useMetrics);
-		
-	}
-
-	@Override
-	protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
-		// read the partitions that belong to the listed topics
-		final List<KafkaTopicPartition> partitions = new ArrayList<>();
-
-		try (KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(this.properties)) {
-			for (final String topic: topics) {
-				// get partitions for each topic
-				List<PartitionInfo> partitionsForTopic = consumer.partitionsFor(topic);
-				// for non existing topics, the list might be null.
-				if (partitionsForTopic != null) {
-					partitions.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic));
-				}
-			}
-		}
-
-		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: {}", partitions.size(), topics);
-
-		if (LOG.isInfoEnabled()) {
-			logPartitionInfo(LOG, partitions);
-		}
-
-		return partitions;
-	}
-
-	// ------------------------------------------------------------------------
-	//  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
-	 */
-	private static List<KafkaTopicPartition> convertToFlinkKafkaTopicPartition(List<PartitionInfo> partitions) {
-		checkNotNull(partitions);
-
-		List<KafkaTopicPartition> ret = new ArrayList<>(partitions.size());
-		for (PartitionInfo pi : partitions) {
-			ret.add(new KafkaTopicPartition(pi.topic(), pi.partition()));
-		}
-		return ret;
-	}
-
-	/**
-	 * Makes sure that the ByteArrayDeserializer is registered in the Kafka properties.
-	 * 
-	 * @param props The Kafka properties to register the serializer in.
-	 */
-	private static void setDeserializer(Properties props) {
-		final String deSerName = ByteArrayDeserializer.class.getCanonicalName();
-
-		Object keyDeSer = props.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
-		Object valDeSer = props.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
-
-		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);
-		}
-
-		props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deSerName);
-		props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deSerName);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
deleted file mode 100644
index 2a3e39d..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-import java.util.Properties;
-
-
-/**
- * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.9.
- *
- * Please note that this producer does not have any reliability guarantees.
- *
- * @param <IN> Type of the messages to write into Kafka.
- */
-public class FlinkKafkaProducer09<IN> extends FlinkKafkaProducerBase<IN> {
-
-	private static final long serialVersionUID = 1L;
-
-	// ------------------- Keyless serialization schema constructors ----------------------
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 *			Comma separated addresses of the brokers
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined (keyless) serialization schema.
-	 */
-	public FlinkKafkaProducer09(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
-		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner<IN>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined (keyless) serialization schema.
-	 * @param producerConfig
-	 * 			Properties with the producer configuration.
-	 */
-	public FlinkKafkaProducer09(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
-		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner<IN>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param topicId The topic to write data to
-	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
-	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
-	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
-	 */
-	public FlinkKafkaProducer09(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
-		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
-
-	}
-
-	// ------------------- Key/Value serialization schema constructors ----------------------
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 *			Comma separated addresses of the brokers
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined serialization schema supporting key/value messages
-	 */
-	public FlinkKafkaProducer09(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
-		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner<IN>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined serialization schema supporting key/value messages
-	 * @param producerConfig
-	 * 			Properties with the producer configuration.
-	 */
-	public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
-		this(topicId, serializationSchema, producerConfig, new FixedPartitioner<IN>());
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
-	 * the topic.
-	 *
-	 * @param topicId The topic to write data to
-	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
-	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
-	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
-	 */
-	public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
-		super(topicId, serializationSchema, producerConfig, customPartitioner);
-	}
-
-	@Override
-	protected void flush() {
-		if (this.producer != null) {
-			producer.flush();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
deleted file mode 100644
index 38ea47c..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-import java.util.Properties;
-
-/**
- * Kafka 0.9 {@link KafkaTableSink} that serializes data in JSON format.
- */
-public class Kafka09JsonTableSink extends KafkaJsonTableSink {
-	/**
-	 * Creates {@link KafkaTableSink} for Kafka 0.9
-	 *
-	 * @param topic topic in Kafka to which table is written
-	 * @param properties properties to connect to Kafka
-	 * @param partitioner Kafka partitioner
-	 */
-	public Kafka09JsonTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner) {
-		super(topic, properties, partitioner);
-	}
-
-	@Override
-	protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties, SerializationSchema<Row> serializationSchema, KafkaPartitioner<Row> partitioner) {
-		return new FlinkKafkaProducer09<>(topic, serializationSchema, properties, partitioner);
-	}
-
-	@Override
-	protected Kafka09JsonTableSink createCopy() {
-		return new Kafka09JsonTableSink(topic, properties, partitioner);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
deleted file mode 100644
index 975ef58..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Kafka {@link StreamTableSource} for Kafka 0.9.
- */
-public class Kafka09JsonTableSource extends KafkaJsonTableSource {
-
-	/**
-	 * Creates a Kafka 0.9 JSON {@link StreamTableSource}.
-	 *
-	 * @param topic      Kafka topic to consume.
-	 * @param properties Properties for the Kafka consumer.
-	 * @param fieldNames Row field names.
-	 * @param fieldTypes Row field types.
-	 */
-	public Kafka09JsonTableSource(
-			String topic,
-			Properties properties,
-			String[] fieldNames,
-			TypeInformation<?>[] fieldTypes) {
-
-		super(topic, properties, fieldNames, fieldTypes);
-	}
-
-	/**
-	 * Creates a Kafka 0.9 JSON {@link StreamTableSource}.
-	 *
-	 * @param topic      Kafka topic to consume.
-	 * @param properties Properties for the Kafka consumer.
-	 * @param fieldNames Row field names.
-	 * @param fieldTypes Row field types.
-	 */
-	public Kafka09JsonTableSource(
-			String topic,
-			Properties properties,
-			String[] fieldNames,
-			Class<?>[] fieldTypes) {
-
-		super(topic, properties, fieldNames, fieldTypes);
-	}
-
-	@Override
-	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
-		return new FlinkKafkaConsumer09<>(topic, deserializationSchema, properties);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
deleted file mode 100644
index 03b5040..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Kafka {@link StreamTableSource} for Kafka 0.9.
- */
-public class Kafka09TableSource extends KafkaTableSource {
-
-	/**
-	 * Creates a Kafka 0.9 {@link StreamTableSource}.
-	 *
-	 * @param topic                 Kafka topic to consume.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param deserializationSchema Deserialization schema to use for Kafka records.
-	 * @param fieldNames            Row field names.
-	 * @param fieldTypes            Row field types.
-	 */
-	public Kafka09TableSource(
-			String topic,
-			Properties properties,
-			DeserializationSchema<Row> deserializationSchema,
-			String[] fieldNames,
-			TypeInformation<?>[] fieldTypes) {
-
-		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
-	}
-
-	/**
-	 * Creates a Kafka 0.9 {@link StreamTableSource}.
-	 *
-	 * @param topic                 Kafka topic to consume.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param deserializationSchema Deserialization schema to use for Kafka records.
-	 * @param fieldNames            Row field names.
-	 * @param fieldTypes            Row field types.
-	 */
-	public Kafka09TableSource(
-			String topic,
-			Properties properties,
-			DeserializationSchema<Row> deserializationSchema,
-			String[] fieldNames,
-			Class<?>[] fieldTypes) {
-
-		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
-	}
-
-	@Override
-	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
-		return new FlinkKafkaConsumer09<>(topic, deserializationSchema, properties);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java
deleted file mode 100644
index e6e3c51..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internal;
-
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-
-import javax.annotation.Nonnull;
-import javax.annotation.concurrent.ThreadSafe;
-import java.io.Closeable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The Handover is a utility to hand over data (a buffer of records) and exception from a
- * <i>producer</i> thread to a <i>consumer</i> thread. It effectively behaves like a
- * "size one blocking queue", with some extras around exception reporting, closing, and
- * waking up thread without {@link Thread#interrupt() interrupting} threads.
- * 
- * <p>This class is used in the Flink Kafka Consumer to hand over data and exceptions between
- * the thread that runs the KafkaConsumer class and the main thread.
- * 
- * <p>The Handover has the notion of "waking up" the producer thread with a {@link WakeupException}
- * rather than a thread interrupt.
- * 
- * <p>The Handover can also be "closed", signalling from one thread to the other that it
- * the thread has terminated.
- */
-@ThreadSafe
-public final class Handover implements Closeable {
-
-	private final Object lock = new Object();
-
-	private ConsumerRecords<byte[], byte[]> next;
-	private Throwable error;
-	private boolean wakeupProducer;
-
-	/**
-	 * Polls the next element from the Handover, possibly blocking until the next element is
-	 * available. This method behaves similar to polling from a blocking queue.
-	 * 
-	 * <p>If an exception was handed in by the producer ({@link #reportError(Throwable)}), then
-	 * that exception is thrown rather than an element being returned.
-	 * 
-	 * @return The next element (buffer of records, never null).
-	 * 
-	 * @throws ClosedException Thrown if the Handover was {@link #close() closed}.
-	 * @throws Exception Rethrows exceptions from the {@link #reportError(Throwable)} method.
-	 */
-	@Nonnull
-	public ConsumerRecords<byte[], byte[]> pollNext() throws Exception {
-		synchronized (lock) {
-			while (next == null && error == null) {
-				lock.wait();
-			}
-
-			ConsumerRecords<byte[], byte[]> n = next;
-			if (n != null) {
-				next = null;
-				lock.notifyAll();
-				return n;
-			}
-			else {
-				ExceptionUtils.rethrowException(error, error.getMessage());
-
-				// this statement cannot be reached since the above method always throws an exception
-				// this is only here to silence the compiler and any warnings
-				return ConsumerRecords.empty(); 
-			}
-		}
-	}
-
-	/**
-	 * Hands over an element from the producer. If the Handover already has an element that was
-	 * not yet picked up by the consumer thread, this call blocks until the consumer picks up that
-	 * previous element.
-	 * 
-	 * <p>This behavior is similar to a "size one" blocking queue.
-	 * 
-	 * @param element The next element to hand over.
-	 * 
-	 * @throws InterruptedException
-	 *                 Thrown, if the thread is interrupted while blocking for the Handover to be empty.
-	 * @throws WakeupException
-	 *                 Thrown, if the {@link #wakeupProducer()} method is called while blocking for
-	 *                 the Handover to be empty.
-	 * @throws ClosedException
-	 *                 Thrown if the Handover was closed or concurrently being closed.
-	 */
-	public void produce(final ConsumerRecords<byte[], byte[]> element)
-			throws InterruptedException, WakeupException, ClosedException {
-
-		checkNotNull(element);
-
-		synchronized (lock) {
-			while (next != null && !wakeupProducer) {
-				lock.wait();
-			}
-
-			wakeupProducer = false;
-
-			// if there is still an element, we must have been woken up
-			if (next != null) {
-				throw new WakeupException();
-			}
-			// if there is no error, then this is open and can accept this element
-			else if (error == null) {
-				next = element;
-				lock.notifyAll();
-			}
-			// an error marks this as closed for the producer
-			else {
-				throw new ClosedException();
-			}
-		}
-	}
-
-	/**
-	 * Reports an exception. The consumer will throw the given exception immediately, if
-	 * it is currently blocked in the {@link #pollNext()} method, or the next time it
-	 * calls that method.
-	 * 
-	 * <p>After this method has been called, no call to either {@link #produce(ConsumerRecords)}
-	 * or {@link #pollNext()} will ever return regularly any more, but will always return
-	 * exceptionally.
-	 * 
-	 * <p>If another exception was already reported, this method does nothing.
-	 * 
-	 * <p>For the producer, the Handover will appear as if it was {@link #close() closed}.
-	 * 
-	 * @param t The exception to report.
-	 */
-	public void reportError(Throwable t) {
-		checkNotNull(t);
-
-		synchronized (lock) {
-			// do not override the initial exception
-			if (error == null) {
-				error = t;
-			}
-			next = null;
-			lock.notifyAll();
-		}
-	}
-
-	/**
-	 * Closes the handover. Both the {@link #produce(ConsumerRecords)} method and the
-	 * {@link #pollNext()} will throw a {@link ClosedException} on any currently blocking and
-	 * future invocations.
-	 * 
-	 * <p>If an exception was previously reported via the {@link #reportError(Throwable)} method,
-	 * that exception will not be overridden. The consumer thread will throw that exception upon
-	 * calling {@link #pollNext()}, rather than the {@code ClosedException}.
-	 */
-	@Override
-	public void close() {
-		synchronized (lock) {
-			next = null;
-			wakeupProducer = false;
-
-			if (error == null) {
-				error = new ClosedException();
-			}
-			lock.notifyAll();
-		}
-	}
-
-	/**
-	 * Wakes the producer thread up. If the producer thread is currently blocked in
-	 * the {@link #produce(ConsumerRecords)} method, it will exit the method throwing
-	 * a {@link WakeupException}.
-	 */
-	public void wakeupProducer() {
-		synchronized (lock) {
-			wakeupProducer = true;
-			lock.notifyAll();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * An exception thrown by the Handover in the {@link #pollNext()} or
-	 * {@link #produce(ConsumerRecords)} method, after the Handover was closed via
-	 * {@link #close()}.
-	 */
-	public static final class ClosedException extends Exception {
-		private static final long serialVersionUID = 1L;
-	}
-
-	/**
-	 * A special exception thrown bv the Handover in the {@link #produce(ConsumerRecords)}
-	 * method when the producer is woken up from a blocking call via {@link #wakeupProducer()}.
-	 */
-	public static final class WakeupException extends Exception {
-		private static final long serialVersionUID = 1L;
-	}
-}


[50/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/EncoderDecoderTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/EncoderDecoderTest.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/EncoderDecoderTest.java
deleted file mode 100644
index c39db15..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/EncoderDecoderTest.java
+++ /dev/null
@@ -1,528 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.avro;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-import org.apache.avro.reflect.ReflectDatumReader;
-import org.apache.avro.reflect.ReflectDatumWriter;
-import org.apache.flink.api.io.avro.generated.Address;
-import org.apache.flink.api.io.avro.generated.Colors;
-import org.apache.flink.api.io.avro.generated.Fixed16;
-import org.apache.flink.api.io.avro.generated.User;
-import org.apache.flink.util.StringUtils;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-/**
- * Tests the {@link DataOutputEncoder} and {@link DataInputDecoder} classes for Avro serialization.
- */
-public class EncoderDecoderTest {
-	@Test
-	public void testComplexStringsDirecty() {
-		try {
-			Random rnd = new Random(349712539451944123L);
-			
-			for (int i = 0; i < 10; i++) {
-				String testString = StringUtils.getRandomString(rnd, 10, 100);
-				
-				ByteArrayOutputStream baos = new ByteArrayOutputStream(512);
-				{
-					DataOutputStream dataOut = new DataOutputStream(baos);
-					DataOutputEncoder encoder = new DataOutputEncoder();
-					encoder.setOut(dataOut);
-					
-					encoder.writeString(testString);
-					dataOut.flush();
-					dataOut.close();
-				}
-				
-				byte[] data = baos.toByteArray();
-				
-				// deserialize
-				{
-					ByteArrayInputStream bais = new ByteArrayInputStream(data);
-					DataInputStream dataIn = new DataInputStream(bais);
-					DataInputDecoder decoder = new DataInputDecoder();
-					decoder.setIn(dataIn);
-	
-					String deserialized = decoder.readString();
-					
-					assertEquals(testString, deserialized);
-				}
-			}
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail("Test failed due to an exception: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testPrimitiveTypes() {
-		
-		testObjectSerialization(new Boolean(true));
-		testObjectSerialization(new Boolean(false));
-		
-		testObjectSerialization(Byte.valueOf((byte) 0));
-		testObjectSerialization(Byte.valueOf((byte) 1));
-		testObjectSerialization(Byte.valueOf((byte) -1));
-		testObjectSerialization(Byte.valueOf(Byte.MIN_VALUE));
-		testObjectSerialization(Byte.valueOf(Byte.MAX_VALUE));
-		
-		testObjectSerialization(Short.valueOf((short) 0));
-		testObjectSerialization(Short.valueOf((short) 1));
-		testObjectSerialization(Short.valueOf((short) -1));
-		testObjectSerialization(Short.valueOf(Short.MIN_VALUE));
-		testObjectSerialization(Short.valueOf(Short.MAX_VALUE));
-		
-		testObjectSerialization(Integer.valueOf(0));
-		testObjectSerialization(Integer.valueOf(1));
-		testObjectSerialization(Integer.valueOf(-1));
-		testObjectSerialization(Integer.valueOf(Integer.MIN_VALUE));
-		testObjectSerialization(Integer.valueOf(Integer.MAX_VALUE));
-		
-		testObjectSerialization(Long.valueOf(0));
-		testObjectSerialization(Long.valueOf(1));
-		testObjectSerialization(Long.valueOf(-1));
-		testObjectSerialization(Long.valueOf(Long.MIN_VALUE));
-		testObjectSerialization(Long.valueOf(Long.MAX_VALUE));
-		
-		testObjectSerialization(Float.valueOf(0));
-		testObjectSerialization(Float.valueOf(1));
-		testObjectSerialization(Float.valueOf(-1));
-		testObjectSerialization(Float.valueOf((float)Math.E));
-		testObjectSerialization(Float.valueOf((float)Math.PI));
-		testObjectSerialization(Float.valueOf(Float.MIN_VALUE));
-		testObjectSerialization(Float.valueOf(Float.MAX_VALUE));
-		testObjectSerialization(Float.valueOf(Float.MIN_NORMAL));
-		testObjectSerialization(Float.valueOf(Float.NaN));
-		testObjectSerialization(Float.valueOf(Float.NEGATIVE_INFINITY));
-		testObjectSerialization(Float.valueOf(Float.POSITIVE_INFINITY));
-		
-		testObjectSerialization(Double.valueOf(0));
-		testObjectSerialization(Double.valueOf(1));
-		testObjectSerialization(Double.valueOf(-1));
-		testObjectSerialization(Double.valueOf(Math.E));
-		testObjectSerialization(Double.valueOf(Math.PI));
-		testObjectSerialization(Double.valueOf(Double.MIN_VALUE));
-		testObjectSerialization(Double.valueOf(Double.MAX_VALUE));
-		testObjectSerialization(Double.valueOf(Double.MIN_NORMAL));
-		testObjectSerialization(Double.valueOf(Double.NaN));
-		testObjectSerialization(Double.valueOf(Double.NEGATIVE_INFINITY));
-		testObjectSerialization(Double.valueOf(Double.POSITIVE_INFINITY));
-		
-		testObjectSerialization("");
-		testObjectSerialization("abcdefg");
-		testObjectSerialization("ab\u1535\u0155xyz\u706F");
-		
-		testObjectSerialization(new SimpleTypes(3637, 54876486548L, (byte) 65, "We're out looking for astronauts", (short) 0x2387, 2.65767523));
-		testObjectSerialization(new SimpleTypes(705608724, -1L, (byte) -65, "Serve me the sky with a big slice of lemon", (short) Byte.MIN_VALUE, 0.0000001));
-	}
-	
-	@Test
-	public void testArrayTypes() {
-		{
-			int[] array = new int[] {1, 2, 3, 4, 5};
-			testObjectSerialization(array);
-		}
-		{
-			long[] array = new long[] {1, 2, 3, 4, 5};
-			testObjectSerialization(array);
-		}
-		{
-			float[] array = new float[] {1, 2, 3, 4, 5};
-			testObjectSerialization(array);
-		}
-		{
-			double[] array = new double[] {1, 2, 3, 4, 5};
-			testObjectSerialization(array);
-		}
-		{
-			String[] array = new String[] {"Oh", "my", "what", "do", "we", "have", "here", "?"};
-			testObjectSerialization(array);
-		}
-	}
-	
-	@Test
-	public void testEmptyArray() {
-		{
-			int[] array = new int[0];
-			testObjectSerialization(array);
-		}
-		{
-			long[] array = new long[0];
-			testObjectSerialization(array);
-		}
-		{
-			float[] array = new float[0];
-			testObjectSerialization(array);
-		}
-		{
-			double[] array = new double[0];
-			testObjectSerialization(array);
-		}
-		{
-			String[] array = new String[0];
-			testObjectSerialization(array);
-		}
-	}
-	
-	@Test
-	public void testObjects() {
-		// simple object containing only primitives
-		{
-			testObjectSerialization(new Book(976243875L, "The Serialization Odysse", 42));
-		}
-		
-		// object with collection
-		{
-			ArrayList<String> list = new ArrayList<String>();
-			list.add("A");
-			list.add("B");
-			list.add("C");
-			list.add("D");
-			list.add("E");
-			
-			testObjectSerialization(new BookAuthor(976243875L, list, "Arno Nym"));
-		}
-		
-		// object with empty collection
-		{
-			ArrayList<String> list = new ArrayList<String>();
-			testObjectSerialization(new BookAuthor(987654321L, list, "The Saurus"));
-		}
-	}
-	
-	@Test
-	public void testNestedObjectsWithCollections() {
-		testObjectSerialization(new ComplexNestedObject2(true));
-	}
-	
-	@Test
-	public void testGeneratedObjectWithNullableFields() {
-		List<CharSequence> strings = Arrays.asList(new CharSequence[] { "These", "strings", "should", "be", "recognizable", "as", "a", "meaningful", "sequence" });
-		List<Boolean> bools = Arrays.asList(true, true, false, false, true, false, true, true);
-		Map<CharSequence, Long> map = new HashMap<CharSequence, Long>();
-		map.put("1", 1L);
-		map.put("2", 2L);
-		map.put("3", 3L);
-
-		byte[] b = new byte[16];
-		new Random().nextBytes(b);
-		Fixed16 f = new Fixed16(b);
-		Address addr = new Address(new Integer(239), "6th Main", "Bangalore",
-				"Karnataka", "560075");
-		User user = new User("Freudenreich", 1337, "macintosh gray",
-				1234567890L, 3.1415926, null, true, strings, bools, null,
-				Colors.GREEN, map, f, new Boolean(true), addr);
-		
-		testObjectSerialization(user);
-	}
-	
-	@Test
-	public void testVarLenCountEncoding() {
-		try {
-			long[] values = new long[] { 0, 1, 2, 3, 4, 0, 574, 45236, 0, 234623462, 23462462346L, 0, 9734028767869761L, 0x7fffffffffffffffL};
-			
-			// write
-			ByteArrayOutputStream baos = new ByteArrayOutputStream(512);
-			{
-				DataOutputStream dataOut = new DataOutputStream(baos);
-				
-				for (long val : values) {
-					DataOutputEncoder.writeVarLongCount(dataOut, val);
-				}
-				
-				dataOut.flush();
-				dataOut.close();
-			}
-			
-			// read
-			{
-				ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-				DataInputStream dataIn = new DataInputStream(bais);
-				
-				for (long val : values) {
-					long read = DataInputDecoder.readVarLongCount(dataIn);
-					assertEquals("Wrong var-len encoded value read.", val, read);
-				}
-			}
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail("Test failed due to an exception: " + e.getMessage());
-		}
-	}
-	
-	private static <X> void testObjectSerialization(X obj) {
-		
-		try {
-			
-			// serialize
-			ByteArrayOutputStream baos = new ByteArrayOutputStream(512);
-			{
-				DataOutputStream dataOut = new DataOutputStream(baos);
-				DataOutputEncoder encoder = new DataOutputEncoder();
-				encoder.setOut(dataOut);
-				
-				@SuppressWarnings("unchecked")
-				Class<X> clazz = (Class<X>) obj.getClass();
-				ReflectDatumWriter<X> writer = new ReflectDatumWriter<X>(clazz);
-				
-				writer.write(obj, encoder);
-				dataOut.flush();
-				dataOut.close();
-			}
-			
-			byte[] data = baos.toByteArray();
-			X result = null;
-			
-			// deserialize
-			{
-				ByteArrayInputStream bais = new ByteArrayInputStream(data);
-				DataInputStream dataIn = new DataInputStream(bais);
-				DataInputDecoder decoder = new DataInputDecoder();
-				decoder.setIn(dataIn);
-
-				@SuppressWarnings("unchecked")
-				Class<X> clazz = (Class<X>) obj.getClass();
-				ReflectDatumReader<X> reader = new ReflectDatumReader<X>(clazz);
-				
-				// create a reuse object if possible, otherwise we have no reuse object 
-				X reuse = null;
-				try {
-					@SuppressWarnings("unchecked")
-					X test = (X) obj.getClass().newInstance();
-					reuse = test;
-				} catch (Throwable t) {}
-				
-				result = reader.read(reuse, decoder);
-			}
-			
-			// check
-			final String message = "Deserialized object is not the same as the original";
-			
-			if (obj.getClass().isArray()) {
-				Class<?> clazz = obj.getClass();
-				if (clazz == byte[].class) {
-					assertArrayEquals(message, (byte[]) obj, (byte[]) result);
-				}
-				else if (clazz == short[].class) {
-					assertArrayEquals(message, (short[]) obj, (short[]) result);
-				}
-				else if (clazz == int[].class) {
-					assertArrayEquals(message, (int[]) obj, (int[]) result);
-				}
-				else if (clazz == long[].class) {
-					assertArrayEquals(message, (long[]) obj, (long[]) result);
-				}
-				else if (clazz == char[].class) {
-					assertArrayEquals(message, (char[]) obj, (char[]) result);
-				}
-				else if (clazz == float[].class) {
-					assertArrayEquals(message, (float[]) obj, (float[]) result, 0.0f);
-				}
-				else if (clazz == double[].class) {
-					assertArrayEquals(message, (double[]) obj, (double[]) result, 0.0);
-				} else {
-					assertArrayEquals(message, (Object[]) obj, (Object[]) result);
-				}
-			} else {
-				assertEquals(message, obj, result);
-			}
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail("Test failed due to an exception: " + e.getMessage());
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//  Test Objects
-	// --------------------------------------------------------------------------------------------
-
-
-	public static final class SimpleTypes {
-		
-		private final int iVal;
-		private final long lVal;
-		private final byte bVal;
-		private final String sVal;
-		private final short rVal;
-		private final double dVal;
-		
-		
-		public SimpleTypes() {
-			this(0, 0, (byte) 0, "", (short) 0, 0);
-		}
-		
-		public SimpleTypes(int iVal, long lVal, byte bVal, String sVal, short rVal, double dVal) {
-			this.iVal = iVal;
-			this.lVal = lVal;
-			this.bVal = bVal;
-			this.sVal = sVal;
-			this.rVal = rVal;
-			this.dVal = dVal;
-		}
-		
-		@Override
-		public boolean equals(Object obj) {
-			if (obj.getClass() == SimpleTypes.class) {
-				SimpleTypes other = (SimpleTypes) obj;
-				
-				return other.iVal == this.iVal &&
-						other.lVal == this.lVal &&
-						other.bVal == this.bVal &&
-						other.sVal.equals(this.sVal) &&
-						other.rVal == this.rVal &&
-						other.dVal == this.dVal;
-				
-			} else {
-				return false;
-			}
-		}
-	}
-	
-	public static class ComplexNestedObject1 {
-		
-		private double doubleValue;
-		
-		private List<String> stringList;
-		
-		public ComplexNestedObject1() {}
-		
-		public ComplexNestedObject1(int offInit) {
-			this.doubleValue = 6293485.6723 + offInit;
-				
-			this.stringList = new ArrayList<String>();
-			this.stringList.add("A" + offInit);
-			this.stringList.add("somewhat" + offInit);
-			this.stringList.add("random" + offInit);
-			this.stringList.add("collection" + offInit);
-			this.stringList.add("of" + offInit);
-			this.stringList.add("strings" + offInit);
-		}
-		
-		@Override
-		public boolean equals(Object obj) {
-			if (obj.getClass() == ComplexNestedObject1.class) {
-				ComplexNestedObject1 other = (ComplexNestedObject1) obj;
-				return other.doubleValue == this.doubleValue && this.stringList.equals(other.stringList);
-			} else {
-				return false;
-			}
-		}
-	}
-	
-	public static class ComplexNestedObject2 {
-		
-		private long longValue;
-		
-		private Map<String, ComplexNestedObject1> theMap;
-		
-		public ComplexNestedObject2() {}
-		
-		public ComplexNestedObject2(boolean init) {
-			this.longValue = 46547;
-				
-			this.theMap = new HashMap<String, ComplexNestedObject1>();
-			this.theMap.put("36354L", new ComplexNestedObject1(43546543));
-			this.theMap.put("785611L", new ComplexNestedObject1(45784568));
-			this.theMap.put("43L", new ComplexNestedObject1(9876543));
-			this.theMap.put("-45687L", new ComplexNestedObject1(7897615));
-			this.theMap.put("1919876876896L", new ComplexNestedObject1(27154));
-			this.theMap.put("-868468468L", new ComplexNestedObject1(546435));
-		}
-		
-		@Override
-		public boolean equals(Object obj) {
-			if (obj.getClass() == ComplexNestedObject2.class) {
-				ComplexNestedObject2 other = (ComplexNestedObject2) obj;
-				return other.longValue == this.longValue && this.theMap.equals(other.theMap);
-			} else {
-				return false;
-			}
-		}
-	}
-	
-	public static class Book {
-
-		private long bookId;
-		private String title;
-		private long authorId;
-
-		public Book() {}
-
-		public Book(long bookId, String title, long authorId) {
-			this.bookId = bookId;
-			this.title = title;
-			this.authorId = authorId;
-		}
-		
-		@Override
-		public boolean equals(Object obj) {
-			if (obj.getClass() == Book.class) {
-				Book other = (Book) obj;
-				return other.bookId == this.bookId && other.authorId == this.authorId && this.title.equals(other.title);
-			} else {
-				return false;
-			}
-		}
-	}
-
-	public static class BookAuthor {
-
-		private long authorId;
-		private List<String> bookTitles;
-		private String authorName;
-
-		public BookAuthor() {}
-
-		public BookAuthor(long authorId, List<String> bookTitles, String authorName) {
-			this.authorId = authorId;
-			this.bookTitles = bookTitles;
-			this.authorName = authorName;
-		}
-		
-		@Override
-		public boolean equals(Object obj) {
-			if (obj.getClass() == BookAuthor.class) {
-				BookAuthor other = (BookAuthor) obj;
-				return other.authorName.equals(this.authorName) && other.authorId == this.authorId &&
-						other.bookTitles.equals(this.bookTitles);
-			} else {
-				return false;
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
deleted file mode 100644
index 1174786..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
+++ /dev/null
@@ -1,219 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.avro.testjar;
-
-// ================================================================================================
-//  This file defines the classes for the AvroExternalJarProgramITCase.
-//  The program is exported into src/test/resources/AvroTestProgram.jar.
-//
-//  THIS FILE MUST STAY FULLY COMMENTED SUCH THAT THE HERE DEFINED CLASSES ARE NOT COMPILED
-//  AND ADDED TO THE test-classes DIRECTORY. OTHERWISE, THE EXTERNAL CLASS LOADING WILL
-//  NOT BE COVERED BY THIS TEST.
-// ================================================================================================
-
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.reflect.ReflectData;
-import org.apache.avro.reflect.ReflectDatumWriter;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.AvroInputFormat;
-import org.apache.flink.core.fs.Path;
-
-public class AvroExternalJarProgram  {
-
-	public static final class Color {
-		
-		private String name;
-		private double saturation;
-		
-		public Color() {
-			name = "";
-			saturation = 1.0;
-		}
-		
-		public Color(String name, double saturation) {
-			this.name = name;
-			this.saturation = saturation;
-		}
-		
-		public String getName() {
-			return name;
-		}
-		
-		public void setName(String name) {
-			this.name = name;
-		}
-		
-		public double getSaturation() {
-			return saturation;
-		}
-		
-		public void setSaturation(double saturation) {
-			this.saturation = saturation;
-		}
-		
-		@Override
-		public String toString() {
-			return name + '(' + saturation + ')';
-		}
-	}
-	
-	public static final class MyUser {
-		
-		private String name;
-		private List<Color> colors;
-		
-		public MyUser() {
-			name = "unknown";
-			colors = new ArrayList<Color>();
-		}
-		
-		public MyUser(String name, List<Color> colors) {
-			this.name = name;
-			this.colors = colors;
-		}
-		
-		public String getName() {
-			return name;
-		}
-		
-		public List<Color> getColors() {
-			return colors;
-		}
-		
-		public void setName(String name) {
-			this.name = name;
-		}
-		
-		public void setColors(List<Color> colors) {
-			this.colors = colors;
-		}
-		
-		@Override
-		public String toString() {
-			return name + " : " + colors;
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public static final class NameExtractor extends RichMapFunction<MyUser, Tuple2<String, MyUser>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<String, MyUser> map(MyUser u) {
-			String namePrefix = u.getName().substring(0, 1);
-			return new Tuple2<String, MyUser>(namePrefix, u);
-		}
-	}
-	
-	public static final class NameGrouper extends RichReduceFunction<Tuple2<String, MyUser>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<String, MyUser> reduce(Tuple2<String, MyUser> val1, Tuple2<String, MyUser> val2) {
-			return val1;
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Test Data
-	// --------------------------------------------------------------------------------------------
-	
-	public static final class Generator {
-		
-		private final Random rnd = new Random(2389756789345689276L);
-		
-		public MyUser nextUser() {
-			return randomUser();
-		}
-		
-		private MyUser randomUser() {
-			
-			int numColors = rnd.nextInt(5);
-			ArrayList<Color> colors = new ArrayList<Color>(numColors);
-			for (int i = 0; i < numColors; i++) {
-				colors.add(new Color(randomString(), rnd.nextDouble()));
-			}
-			
-			return new MyUser(randomString(), colors);
-		}
-		
-		private String randomString() {
-			char[] c = new char[this.rnd.nextInt(20) + 5];
-			
-			for (int i = 0; i < c.length; i++) {
-				c[i] = (char) (this.rnd.nextInt(150) + 40);
-			}
-			
-			return new String(c);
-		}
-	}
-	
-	public static void writeTestData(File testFile, int numRecords) throws IOException {
-		
-		DatumWriter<MyUser> userDatumWriter = new ReflectDatumWriter<MyUser>(MyUser.class);
-		DataFileWriter<MyUser> dataFileWriter = new DataFileWriter<MyUser>(userDatumWriter);
-		
-		dataFileWriter.create(ReflectData.get().getSchema(MyUser.class), testFile);
-		
-		
-		Generator generator = new Generator();
-		
-		for (int i = 0; i < numRecords; i++) {
-			MyUser user = generator.nextUser();
-			dataFileWriter.append(user);
-		}
-		
-		dataFileWriter.close();
-	}
-
-//	public static void main(String[] args) throws Exception {
-//		String testDataFile = new File("src/test/resources/testdata.avro").getAbsolutePath();
-//		writeTestData(new File(testDataFile), 50);
-//	}
-	
-	public static void main(String[] args) throws Exception {
-		String inputPath = args[0];
-		
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		DataSet<MyUser> input = env.createInput(new AvroInputFormat<MyUser>(new Path(inputPath), MyUser.class));
-	
-		DataSet<Tuple2<String, MyUser>> result = input.map(new NameExtractor()).groupBy(0).reduce(new NameGrouper());
-		
-		result.output(new DiscardingOutputFormat<Tuple2<String,MyUser>>());
-		env.execute();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
deleted file mode 100644
index f33f433..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.api.io.avro;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.io.avro.generated.User;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.io.AvroInputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.File;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-@RunWith(Parameterized.class)
-public class AvroPojoTest extends MultipleProgramsTestBase {
-	public AvroPojoTest(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	private File inFile;
-	private String resultPath;
-	private String expected;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Before
-	public void before() throws Exception{
-		resultPath = tempFolder.newFile().toURI().toString();
-		inFile = tempFolder.newFile();
-		AvroRecordInputFormatTest.writeTestFile(inFile);
-	}
-
-	@After
-	public void after() throws Exception{
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-
-	@Test
-	public void testSimpleAvroRead() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Path in = new Path(inFile.getAbsoluteFile().toURI());
-
-		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
-		DataSet<User> usersDS = env.createInput(users)
-				// null map type because the order changes in different JVMs (hard to test)
-		.map(new MapFunction<User, User>() {
-			@Override
-			public User map(User value) throws Exception {
-				value.setTypeMap(null);
-				return value;
-			}
-		});
-
-		usersDS.writeAsText(resultPath);
-
-		env.execute("Simple Avro read job");
-
-
-		expected = "{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null, \"type_long_test\": null, \"type_double_test\": 123.45, \"type_null_test\": null, \"type_bool_test\": true, \"type_array_string\": [\"ELEMENT 1\", \"ELEMENT 2\"], \"type_array_boolean\": [true, false], \"type_nullable_array\": null, \"type_enum\": \"GREEN\", \"type_map\": null, \"type_fixed\": null, \"type_union\": null, \"type_nested\": {\"num\": 239, \"street\": \"Baker Street\", \"city\": \"London\", \"state\": \"London\", \"zip\": \"NW1 6XE\"}}\n" +
-					"{\"name\": \"Charlie\", \"favorite_number\": null, \"favorite_color\": \"blue\", \"type_long_test\": 1337, \"type_double_test\": 1.337, \"type_null_test\": null, \"type_bool_test\": false, \"type_array_string\": [], \"type_array_boolean\": [], \"type_nullable_array\": null, \"type_enum\": \"RED\", \"type_map\": null, \"type_fixed\": null, \"type_union\": null, \"type_nested\": {\"num\": 239, \"street\": \"Baker Street\", \"city\": \"London\", \"state\": \"London\", \"zip\": \"NW1 6XE\"}}\n";
-	}
-
-	@Test
-	public void testSerializeWithAvro() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableForceAvro();
-		Path in = new Path(inFile.getAbsoluteFile().toURI());
-
-		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
-		DataSet<User> usersDS = env.createInput(users)
-				// null map type because the order changes in different JVMs (hard to test)
-				.map(new MapFunction<User, User>() {
-					@Override
-					public User map(User value) throws Exception {
-						Map<CharSequence, Long> ab = new HashMap<CharSequence, Long>(1);
-						ab.put("hehe", 12L);
-						value.setTypeMap(ab);
-						return value;
-					}
-				});
-
-		usersDS.writeAsText(resultPath);
-
-		env.execute("Simple Avro read job");
-
-
-		expected = "{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null, \"type_long_test\": null, \"type_double_test\": 123.45, \"type_null_test\": null, \"type_bool_test\": true, \"type_array_string\": [\"ELEMENT 1\", \"ELEMENT 2\"], \"type_array_boolean\": [true, false], \"type_nullable_array\": null, \"type_enum\": \"GREEN\", \"type_map\": {\"hehe\": 12}, \"type_fixed\": null, \"type_union\": null, \"type_nested\": {\"num\": 239, \"street\": \"Baker Street\", \"city\": \"London\", \"state\": \"London\", \"zip\": \"NW1 6XE\"}}\n" +
-					"{\"name\": \"Charlie\", \"favorite_number\": null, \"favorite_color\": \"blue\", \"type_long_test\": 1337, \"type_double_test\": 1.337, \"type_null_test\": null, \"type_bool_test\": false, \"type_array_string\": [], \"type_array_boolean\": [], \"type_nullable_array\": null, \"type_enum\": \"RED\", \"type_map\": {\"hehe\": 12}, \"type_fixed\": null, \"type_union\": null, \"type_nested\": {\"num\": 239, \"street\": \"Baker Street\", \"city\": \"London\", \"state\": \"London\", \"zip\": \"NW1 6XE\"}}\n";
-
-	}
-
-	@Test
-	public void testKeySelection() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableObjectReuse();
-		Path in = new Path(inFile.getAbsoluteFile().toURI());
-
-		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
-		DataSet<User> usersDS = env.createInput(users);
-
-		DataSet<Tuple2<String, Integer>> res = usersDS.groupBy("name").reduceGroup(new GroupReduceFunction<User, Tuple2<String, Integer>>() {
-			@Override
-			public void reduce(Iterable<User> values, Collector<Tuple2<String, Integer>> out) throws Exception {
-				for (User u : values) {
-					out.collect(new Tuple2<String, Integer>(u.getName().toString(), 1));
-				}
-			}
-		});
-		res.writeAsText(resultPath);
-		env.execute("Avro Key selection");
-
-
-		expected = "(Alyssa,1)\n(Charlie,1)\n";
-	}
-
-	@Test
-	public void testWithAvroGenericSer() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableForceAvro();
-		Path in = new Path(inFile.getAbsoluteFile().toURI());
-
-		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
-		DataSet<User> usersDS = env.createInput(users);
-
-		DataSet<Tuple2<String, Integer>> res = usersDS.groupBy(new KeySelector<User, String>() {
-			@Override
-			public String getKey(User value) throws Exception {
-				return String.valueOf(value.getName());
-			}
-		}).reduceGroup(new GroupReduceFunction<User, Tuple2<String, Integer>>() {
-			@Override
-			public void reduce(Iterable<User> values, Collector<Tuple2<String, Integer>> out) throws Exception {
-				for(User u : values) {
-					out.collect(new Tuple2<String, Integer>(u.getName().toString(), 1));
-				}
-			}
-		});
-
-		res.writeAsText(resultPath);
-		env.execute("Avro Key selection");
-
-
-		expected = "(Charlie,1)\n(Alyssa,1)\n";
-	}
-
-	@Test
-	public void testWithKryoGenericSer() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableForceKryo();
-		Path in = new Path(inFile.getAbsoluteFile().toURI());
-
-		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
-		DataSet<User> usersDS = env.createInput(users);
-
-		DataSet<Tuple2<String, Integer>> res = usersDS.groupBy(new KeySelector<User, String>() {
-			@Override
-			public String getKey(User value) throws Exception {
-				return String.valueOf(value.getName());
-			}
-		}).reduceGroup(new GroupReduceFunction<User, Tuple2<String, Integer>>() {
-			@Override
-			public void reduce(Iterable<User> values, Collector<Tuple2<String, Integer>> out) throws Exception {
-				for (User u : values) {
-					out.collect(new Tuple2<String, Integer>(u.getName().toString(), 1));
-				}
-			}
-		});
-
-		res.writeAsText(resultPath);
-		env.execute("Avro Key selection");
-
-
-		expected = "(Charlie,1)\n(Alyssa,1)\n";
-	}
-
-	/**
-	 * Test some know fields for grouping on
-	 */
-	@Test
-	public void testAllFields() throws Exception {
-		for(String fieldName : Arrays.asList("name", "type_enum", "type_double_test")) {
-			testField(fieldName);
-		}
-	}
-
-	private void testField(final String fieldName) throws Exception {
-		before();
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Path in = new Path(inFile.getAbsoluteFile().toURI());
-
-		AvroInputFormat<User> users = new AvroInputFormat<User>(in, User.class);
-		DataSet<User> usersDS = env.createInput(users);
-
-		DataSet<Object> res = usersDS.groupBy(fieldName).reduceGroup(new GroupReduceFunction<User, Object>() {
-			@Override
-			public void reduce(Iterable<User> values, Collector<Object> out) throws Exception {
-				for(User u : values) {
-					out.collect(u.get(fieldName));
-				}
-			}
-		});
-		res.writeAsText(resultPath);
-		env.execute("Simple Avro read job");
-
-		// test if automatic registration of the Types worked
-		ExecutionConfig ec = env.getConfig();
-		Assert.assertTrue(ec.getRegisteredKryoTypes().contains(org.apache.flink.api.io.avro.generated.Fixed16.class));
-
-		if(fieldName.equals("name")) {
-			expected = "Alyssa\nCharlie";
-		} else if(fieldName.equals("type_enum")) {
-			expected = "GREEN\nRED\n";
-		} else if(fieldName.equals("type_double_test")) {
-			expected = "123.45\n1.337\n";
-		} else {
-			Assert.fail("Unknown field");
-		}
-
-		after();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
deleted file mode 100644
index 91a9612..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
+++ /dev/null
@@ -1,458 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.avro;
-
-import static org.junit.Assert.*;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.util.*;
-
-import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileReader;
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.file.FileReader;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.avro.specific.SpecificDatumWriter;
-import org.apache.avro.util.Utf8;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.io.avro.generated.Address;
-import org.apache.flink.api.io.avro.generated.Colors;
-import org.apache.flink.api.io.avro.generated.User;
-import org.apache.flink.api.java.io.AvroInputFormat;
-import org.apache.flink.api.java.typeutils.AvroTypeInfo;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test the avro input format.
- * (The testcase is mostly the getting started tutorial of avro)
- * http://avro.apache.org/docs/current/gettingstartedjava.html
- */
-public class AvroRecordInputFormatTest {
-	
-	public File testFile;
-	
-	final static String TEST_NAME = "Alyssa";
-	
-	final static String TEST_ARRAY_STRING_1 = "ELEMENT 1";
-	final static String TEST_ARRAY_STRING_2 = "ELEMENT 2";
-	
-	final static boolean TEST_ARRAY_BOOLEAN_1 = true;
-	final static boolean TEST_ARRAY_BOOLEAN_2 = false;
-	
-	final static Colors TEST_ENUM_COLOR = Colors.GREEN;
-	
-	final static String TEST_MAP_KEY1 = "KEY 1";
-	final static long TEST_MAP_VALUE1 = 8546456L;
-	final static String TEST_MAP_KEY2 = "KEY 2";
-	final static long TEST_MAP_VALUE2 = 17554L;
-	
-	final static int TEST_NUM = 239;
-	final static String TEST_STREET = "Baker Street";
-	final static String TEST_CITY = "London";
-	final static String TEST_STATE = "London";
-	final static String TEST_ZIP = "NW1 6XE";
-	
-
-	private Schema userSchema = new User().getSchema();
-
-
-	public static void writeTestFile(File testFile) throws IOException {
-		ArrayList<CharSequence> stringArray = new ArrayList<CharSequence>();
-		stringArray.add(TEST_ARRAY_STRING_1);
-		stringArray.add(TEST_ARRAY_STRING_2);
-
-		ArrayList<Boolean> booleanArray = new ArrayList<Boolean>();
-		booleanArray.add(TEST_ARRAY_BOOLEAN_1);
-		booleanArray.add(TEST_ARRAY_BOOLEAN_2);
-
-		HashMap<CharSequence, Long> longMap = new HashMap<CharSequence, Long>();
-		longMap.put(TEST_MAP_KEY1, TEST_MAP_VALUE1);
-		longMap.put(TEST_MAP_KEY2, TEST_MAP_VALUE2);
-		
-		Address addr = new Address();
-		addr.setNum(TEST_NUM);
-		addr.setStreet(TEST_STREET);
-		addr.setCity(TEST_CITY);
-		addr.setState(TEST_STATE);
-		addr.setZip(TEST_ZIP);
-
-
-		User user1 = new User();
-
-		user1.setName(TEST_NAME);
-		user1.setFavoriteNumber(256);
-		user1.setTypeDoubleTest(123.45d);
-		user1.setTypeBoolTest(true);
-		user1.setTypeArrayString(stringArray);
-		user1.setTypeArrayBoolean(booleanArray);
-		user1.setTypeEnum(TEST_ENUM_COLOR);
-		user1.setTypeMap(longMap);
-		user1.setTypeNested(addr);
-
-		// Construct via builder
-		User user2 = User.newBuilder()
-				.setName("Charlie")
-				.setFavoriteColor("blue")
-				.setFavoriteNumber(null)
-				.setTypeBoolTest(false)
-				.setTypeDoubleTest(1.337d)
-				.setTypeNullTest(null)
-				.setTypeLongTest(1337L)
-				.setTypeArrayString(new ArrayList<CharSequence>())
-				.setTypeArrayBoolean(new ArrayList<Boolean>())
-				.setTypeNullableArray(null)
-				.setTypeEnum(Colors.RED)
-				.setTypeMap(new HashMap<CharSequence, Long>())
-				.setTypeFixed(null)
-				.setTypeUnion(null)
-				.setTypeNested(
-						Address.newBuilder().setNum(TEST_NUM).setStreet(TEST_STREET)
-								.setCity(TEST_CITY).setState(TEST_STATE).setZip(TEST_ZIP)
-								.build())
-				.build();
-		DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
-		DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
-		dataFileWriter.create(user1.getSchema(), testFile);
-		dataFileWriter.append(user1);
-		dataFileWriter.append(user2);
-		dataFileWriter.close();
-	}
-	@Before
-	public void createFiles() throws IOException {
-		testFile = File.createTempFile("AvroInputFormatTest", null);
-		writeTestFile(testFile);
-	}
-
-
-	/**
-	 * Test if the AvroInputFormat is able to properly read data from an avro file.
-	 * @throws IOException
-	 */
-	@Test
-	public void testDeserialisation() throws IOException {
-		Configuration parameters = new Configuration();
-		
-		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
-		
-		format.configure(parameters);
-		FileInputSplit[] splits = format.createInputSplits(1);
-		assertEquals(splits.length, 1);
-		format.open(splits[0]);
-		
-		User u = format.nextRecord(null);
-		assertNotNull(u);
-		
-		String name = u.getName().toString();
-		assertNotNull("empty record", name);
-		assertEquals("name not equal", TEST_NAME, name);
-		
-		// check arrays
-		List<CharSequence> sl = u.getTypeArrayString();
-		assertEquals("element 0 not equal", TEST_ARRAY_STRING_1, sl.get(0).toString());
-		assertEquals("element 1 not equal", TEST_ARRAY_STRING_2, sl.get(1).toString());
-		
-		List<Boolean> bl = u.getTypeArrayBoolean();
-		assertEquals("element 0 not equal", TEST_ARRAY_BOOLEAN_1, bl.get(0));
-		assertEquals("element 1 not equal", TEST_ARRAY_BOOLEAN_2, bl.get(1));
-		
-		// check enums
-		Colors enumValue = u.getTypeEnum();
-		assertEquals("enum not equal", TEST_ENUM_COLOR, enumValue);
-		
-		// check maps
-		Map<CharSequence, Long> lm = u.getTypeMap();
-		assertEquals("map value of key 1 not equal", TEST_MAP_VALUE1, lm.get(new Utf8(TEST_MAP_KEY1)).longValue());
-		assertEquals("map value of key 2 not equal", TEST_MAP_VALUE2, lm.get(new Utf8(TEST_MAP_KEY2)).longValue());
-		
-		assertFalse("expecting second element", format.reachedEnd());
-		assertNotNull("expecting second element", format.nextRecord(u));
-		
-		assertNull(format.nextRecord(u));
-		assertTrue(format.reachedEnd());
-		
-		format.close();
-	}
-
-	/**
-	 * Test if the AvroInputFormat is able to properly read data from an avro file.
-	 * @throws IOException
-	 */
-	@Test
-	public void testDeserialisationReuseAvroRecordFalse() throws IOException {
-		Configuration parameters = new Configuration();
-		
-		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
-		format.setReuseAvroValue(false);
-		
-		format.configure(parameters);
-		FileInputSplit[] splits = format.createInputSplits(1);
-		assertEquals(splits.length, 1);
-		format.open(splits[0]);
-		
-		User u = format.nextRecord(null);
-		assertNotNull(u);
-		
-		String name = u.getName().toString();
-		assertNotNull("empty record", name);
-		assertEquals("name not equal", TEST_NAME, name);
-		
-		// check arrays
-		List<CharSequence> sl = u.getTypeArrayString();
-		assertEquals("element 0 not equal", TEST_ARRAY_STRING_1, sl.get(0).toString());
-		assertEquals("element 1 not equal", TEST_ARRAY_STRING_2, sl.get(1).toString());
-		
-		List<Boolean> bl = u.getTypeArrayBoolean();
-		assertEquals("element 0 not equal", TEST_ARRAY_BOOLEAN_1, bl.get(0));
-		assertEquals("element 1 not equal", TEST_ARRAY_BOOLEAN_2, bl.get(1));
-		
-		// check enums
-		Colors enumValue = u.getTypeEnum();
-		assertEquals("enum not equal", TEST_ENUM_COLOR, enumValue);
-		
-		// check maps
-		Map<CharSequence, Long> lm = u.getTypeMap();
-		assertEquals("map value of key 1 not equal", TEST_MAP_VALUE1, lm.get(new Utf8(TEST_MAP_KEY1)).longValue());
-		assertEquals("map value of key 2 not equal", TEST_MAP_VALUE2, lm.get(new Utf8(TEST_MAP_KEY2)).longValue());
-		
-		assertFalse("expecting second element", format.reachedEnd());
-		assertNotNull("expecting second element", format.nextRecord(u));
-		
-		assertNull(format.nextRecord(u));
-		assertTrue(format.reachedEnd());
-		
-		format.close();
-	}
-
-	/**
-	 * Test if the Flink serialization is able to properly process GenericData.Record types.
-	 * Usually users of Avro generate classes (POJOs) from Avro schemas.
-	 * However, if generated classes are not available, one can also use GenericData.Record.
-	 * It is an untyped key-value record which is using a schema to validate the correctness of the data.
-	 *
-	 * It is not recommended to use GenericData.Record with Flink. Use generated POJOs instead.
-	 */
-	@Test
-	public void testDeserializeToGenericType() throws IOException {
-		DatumReader<GenericData.Record> datumReader = new GenericDatumReader<>(userSchema);
-
-		try (FileReader<GenericData.Record> dataFileReader = DataFileReader.openReader(testFile, datumReader)) {
-			// initialize Record by reading it from disk (thats easier than creating it by hand)
-			GenericData.Record rec = new GenericData.Record(userSchema);
-			dataFileReader.next(rec);
-			
-			// check if record has been read correctly
-			assertNotNull(rec);
-			assertEquals("name not equal", TEST_NAME, rec.get("name").toString());
-			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), rec.get("type_enum").toString());
-			assertEquals(null, rec.get("type_long_test")); // it is null for the first record.
-
-			// now serialize it with our framework:
-			TypeInformation<GenericData.Record> te = TypeExtractor.createTypeInfo(GenericData.Record.class);
-
-			ExecutionConfig ec = new ExecutionConfig();
-			Assert.assertEquals(GenericTypeInfo.class, te.getClass());
-			
-			Serializers.recursivelyRegisterType(te.getTypeClass(), ec, new HashSet<Class<?>>());
-
-			TypeSerializer<GenericData.Record> tser = te.createSerializer(ec);
-			Assert.assertEquals(1, ec.getDefaultKryoSerializerClasses().size());
-			Assert.assertTrue(
-					ec.getDefaultKryoSerializerClasses().containsKey(Schema.class) &&
-							ec.getDefaultKryoSerializerClasses().get(Schema.class).equals(Serializers.AvroSchemaSerializer.class));
-
-			ByteArrayOutputStream out = new ByteArrayOutputStream();
-			try (DataOutputViewStreamWrapper outView = new DataOutputViewStreamWrapper(out)) {
-				tser.serialize(rec, outView);
-			}
-
-			GenericData.Record newRec;
-			try (DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(
-					new ByteArrayInputStream(out.toByteArray())))
-			{
-				newRec = tser.deserialize(inView);
-			}
-
-			// check if it is still the same
-			assertNotNull(newRec);
-			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), newRec.get("type_enum").toString());
-			assertEquals("name not equal", TEST_NAME, newRec.get("name").toString());
-			assertEquals(null, newRec.get("type_long_test"));
-		}
-	}
-		
-	/**
-	 * This test validates proper serialization with specific (generated POJO) types.
-	 */
-	@Test
-	public void testDeserializeToSpecificType() throws IOException {
-
-		DatumReader<User> datumReader = new SpecificDatumReader<User>(userSchema);
-
-		try (FileReader<User> dataFileReader = DataFileReader.openReader(testFile, datumReader)) {
-			User rec = dataFileReader.next();
-
-			// check if record has been read correctly
-			assertNotNull(rec);
-			assertEquals("name not equal", TEST_NAME, rec.get("name").toString());
-			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), rec.get("type_enum").toString());
-
-			// now serialize it with our framework:
-			ExecutionConfig ec = new ExecutionConfig();
-			TypeInformation<User> te = TypeExtractor.createTypeInfo(User.class);
-
-			Assert.assertEquals(AvroTypeInfo.class, te.getClass());
-			TypeSerializer<User> tser = te.createSerializer(ec);
-
-			ByteArrayOutputStream out = new ByteArrayOutputStream();
-			try (DataOutputViewStreamWrapper outView = new DataOutputViewStreamWrapper(out)) {
-				tser.serialize(rec, outView);
-			}
-
-			User newRec;
-			try (DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(
-					new ByteArrayInputStream(out.toByteArray())))
-			{
-				newRec = tser.deserialize(inView);
-			}
-
-			// check if it is still the same
-			assertNotNull(newRec);
-			assertEquals("name not equal", TEST_NAME, newRec.getName().toString());
-			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), newRec.getTypeEnum().toString());
-		}
-	}
-
-	/**
-	 * Test if the AvroInputFormat is able to properly read data from an Avro
-	 * file as a GenericRecord.
-	 * 
-	 * @throws IOException,
-	 *             if there is an exception
-	 */
-	@Test
-	public void testDeserialisationGenericRecord() throws IOException {
-		Configuration parameters = new Configuration();
-
-		AvroInputFormat<GenericRecord> format = new AvroInputFormat<GenericRecord>(new Path(testFile.getAbsolutePath()),
-				GenericRecord.class);
-
-		doTestDeserializationGenericRecord(format, parameters);
-	}
-
-	/**
-	 * Helper method to test GenericRecord serialisation
-	 * 
-	 * @param format
-	 *            the format to test
-	 * @param parameters
-	 *            the configuration to use
-	 * @throws IOException
-	 *             thrown id there is a issue
-	 */
-	@SuppressWarnings("unchecked")
-	private void doTestDeserializationGenericRecord(final AvroInputFormat<GenericRecord> format,
-			final Configuration parameters) throws IOException {
-		try {
-			format.configure(parameters);
-			FileInputSplit[] splits = format.createInputSplits(1);
-			assertEquals(splits.length, 1);
-			format.open(splits[0]);
-
-			GenericRecord u = format.nextRecord(null);
-			assertNotNull(u);
-			assertEquals("The schemas should be equal", userSchema, u.getSchema());
-
-			String name = u.get("name").toString();
-			assertNotNull("empty record", name);
-			assertEquals("name not equal", TEST_NAME, name);
-
-			// check arrays
-			List<CharSequence> sl = (List<CharSequence>) u.get("type_array_string");
-			assertEquals("element 0 not equal", TEST_ARRAY_STRING_1, sl.get(0).toString());
-			assertEquals("element 1 not equal", TEST_ARRAY_STRING_2, sl.get(1).toString());
-
-			List<Boolean> bl = (List<Boolean>) u.get("type_array_boolean");
-			assertEquals("element 0 not equal", TEST_ARRAY_BOOLEAN_1, bl.get(0));
-			assertEquals("element 1 not equal", TEST_ARRAY_BOOLEAN_2, bl.get(1));
-
-			// check enums
-			GenericData.EnumSymbol enumValue = (GenericData.EnumSymbol) u.get("type_enum");
-			assertEquals("enum not equal", TEST_ENUM_COLOR.toString(), enumValue.toString());
-
-			// check maps
-			Map<CharSequence, Long> lm = (Map<CharSequence, Long>) u.get("type_map");
-			assertEquals("map value of key 1 not equal", TEST_MAP_VALUE1, lm.get(new Utf8(TEST_MAP_KEY1)).longValue());
-			assertEquals("map value of key 2 not equal", TEST_MAP_VALUE2, lm.get(new Utf8(TEST_MAP_KEY2)).longValue());
-
-			assertFalse("expecting second element", format.reachedEnd());
-			assertNotNull("expecting second element", format.nextRecord(u));
-
-			assertNull(format.nextRecord(u));
-			assertTrue(format.reachedEnd());
-		} finally {
-			format.close();
-		}
-	}
-
-	/**
-	 * Test if the AvroInputFormat is able to properly read data from an avro
-	 * file as a GenericRecord
-	 * 
-	 * @throws IOException,
-	 *             if there is an error
-	 */
-	@Test
-	public void testDeserialisationGenericRecordReuseAvroValueFalse() throws IOException {
-		Configuration parameters = new Configuration();
-
-		AvroInputFormat<GenericRecord> format = new AvroInputFormat<GenericRecord>(new Path(testFile.getAbsolutePath()),
-				GenericRecord.class);
-		format.configure(parameters);
-		format.setReuseAvroValue(false);
-
-		doTestDeserializationGenericRecord(format, parameters);
-	}
-
-	@After
-	public void deleteFiles() {
-		testFile.delete();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroSplittableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroSplittableInputFormatTest.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroSplittableInputFormatTest.java
deleted file mode 100644
index 37a83d1..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroSplittableInputFormatTest.java
+++ /dev/null
@@ -1,326 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.io.avro;
-
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.specific.SpecificDatumWriter;
-import org.apache.flink.api.io.avro.generated.Address;
-import org.apache.flink.api.io.avro.generated.Colors;
-import org.apache.flink.api.io.avro.generated.Fixed16;
-import org.apache.flink.api.io.avro.generated.User;
-import org.apache.flink.api.java.io.AvroInputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.Path;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Random;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test the avro input format.
- * (The testcase is mostly the getting started tutorial of avro)
- * http://avro.apache.org/docs/current/gettingstartedjava.html
- */
-public class AvroSplittableInputFormatTest {
-	
-	private File testFile;
-	
-	final static String TEST_NAME = "Alyssa";
-	
-	final static String TEST_ARRAY_STRING_1 = "ELEMENT 1";
-	final static String TEST_ARRAY_STRING_2 = "ELEMENT 2";
-	
-	final static boolean TEST_ARRAY_BOOLEAN_1 = true;
-	final static boolean TEST_ARRAY_BOOLEAN_2 = false;
-	
-	final static Colors TEST_ENUM_COLOR = Colors.GREEN;
-	
-	final static String TEST_MAP_KEY1 = "KEY 1";
-	final static long TEST_MAP_VALUE1 = 8546456L;
-	final static String TEST_MAP_KEY2 = "KEY 2";
-	final static long TEST_MAP_VALUE2 = 17554L;
-
-	final static Integer TEST_NUM = new Integer(239);
-	final static String TEST_STREET = "Baker Street";
-	final static String TEST_CITY = "London";
-	final static String TEST_STATE = "London";
-	final static String TEST_ZIP = "NW1 6XE";
-	
-	final static int NUM_RECORDS = 5000;
-
-	@Before
-	public void createFiles() throws IOException {
-		testFile = File.createTempFile("AvroSplittableInputFormatTest", null);
-		
-		ArrayList<CharSequence> stringArray = new ArrayList<CharSequence>();
-		stringArray.add(TEST_ARRAY_STRING_1);
-		stringArray.add(TEST_ARRAY_STRING_2);
-		
-		ArrayList<Boolean> booleanArray = new ArrayList<Boolean>();
-		booleanArray.add(TEST_ARRAY_BOOLEAN_1);
-		booleanArray.add(TEST_ARRAY_BOOLEAN_2);
-		
-		HashMap<CharSequence, Long> longMap = new HashMap<CharSequence, Long>();
-		longMap.put(TEST_MAP_KEY1, TEST_MAP_VALUE1);
-		longMap.put(TEST_MAP_KEY2, TEST_MAP_VALUE2);
-		
-		Address addr = new Address();
-		addr.setNum(new Integer(TEST_NUM));
-		addr.setStreet(TEST_STREET);
-		addr.setCity(TEST_CITY);
-		addr.setState(TEST_STATE);
-		addr.setZip(TEST_ZIP);
-		
-		
-		User user1 = new User();
-		user1.setName(TEST_NAME);
-		user1.setFavoriteNumber(256);
-		user1.setTypeDoubleTest(123.45d);
-		user1.setTypeBoolTest(true);
-		user1.setTypeArrayString(stringArray);
-		user1.setTypeArrayBoolean(booleanArray);
-		user1.setTypeEnum(TEST_ENUM_COLOR);
-		user1.setTypeMap(longMap);
-		user1.setTypeNested(addr);
-		
-		// Construct via builder
-		User user2 = User.newBuilder()
-		             .setName(TEST_NAME)
-		             .setFavoriteColor("blue")
-		             .setFavoriteNumber(null)
-		             .setTypeBoolTest(false)
-		             .setTypeDoubleTest(1.337d)
-		             .setTypeNullTest(null)
-		             .setTypeLongTest(1337L)
-		             .setTypeArrayString(new ArrayList<CharSequence>())
-		             .setTypeArrayBoolean(new ArrayList<Boolean>())
-		             .setTypeNullableArray(null)
-		             .setTypeEnum(Colors.RED)
-		             .setTypeMap(new HashMap<CharSequence, Long>())
-					 .setTypeFixed(new Fixed16())
-					 .setTypeUnion(123L)
-				.setTypeNested(
-						Address.newBuilder().setNum(TEST_NUM).setStreet(TEST_STREET)
-								.setCity(TEST_CITY).setState(TEST_STATE).setZip(TEST_ZIP)
-								.build())
-
-		             .build();
-		DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
-		DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
-		dataFileWriter.create(user1.getSchema(), testFile);
-		dataFileWriter.append(user1);
-		dataFileWriter.append(user2);
-
-		Random rnd = new Random(1337);
-		for(int i = 0; i < NUM_RECORDS -2 ; i++) {
-			User user = new User();
-			user.setName(TEST_NAME + rnd.nextInt());
-			user.setFavoriteNumber(rnd.nextInt());
-			user.setTypeDoubleTest(rnd.nextDouble());
-			user.setTypeBoolTest(true);
-			user.setTypeArrayString(stringArray);
-			user.setTypeArrayBoolean(booleanArray);
-			user.setTypeEnum(TEST_ENUM_COLOR);
-			user.setTypeMap(longMap);
-			Address address = new Address();
-			address.setNum(new Integer(TEST_NUM));
-			address.setStreet(TEST_STREET);
-			address.setCity(TEST_CITY);
-			address.setState(TEST_STATE);
-			address.setZip(TEST_ZIP);
-			user.setTypeNested(address);
-
-			dataFileWriter.append(user);
-		}
-		dataFileWriter.close();
-	}
-	
-	@Test
-	public void testSplittedIF() throws IOException {
-		Configuration parameters = new Configuration();
-		
-		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
-
-		format.configure(parameters);
-		FileInputSplit[] splits = format.createInputSplits(4);
-		assertEquals(splits.length, 4);
-		int elements = 0;
-		int elementsPerSplit[] = new int[4];
-		for(int i = 0; i < splits.length; i++) {
-			format.open(splits[i]);
-			while(!format.reachedEnd()) {
-				User u = format.nextRecord(null);
-				Assert.assertTrue(u.getName().toString().startsWith(TEST_NAME));
-				elements++;
-				elementsPerSplit[i]++;
-			}
-			format.close();
-		}
-
-		Assert.assertEquals(1539, elementsPerSplit[0]);
-		Assert.assertEquals(1026, elementsPerSplit[1]);
-		Assert.assertEquals(1539, elementsPerSplit[2]);
-		Assert.assertEquals(896, elementsPerSplit[3]);
-		Assert.assertEquals(NUM_RECORDS, elements);
-		format.close();
-	}
-
-	@Test
-	public void testAvroRecoveryWithFailureAtStart() throws Exception {
-		final int recordsUntilCheckpoint = 132;
-
-		Configuration parameters = new Configuration();
-
-		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
-		format.configure(parameters);
-
-		FileInputSplit[] splits = format.createInputSplits(4);
-		assertEquals(splits.length, 4);
-
-		int elements = 0;
-		int elementsPerSplit[] = new int[4];
-		for(int i = 0; i < splits.length; i++) {
-			format.reopen(splits[i], format.getCurrentState());
-			while(!format.reachedEnd()) {
-				User u = format.nextRecord(null);
-				Assert.assertTrue(u.getName().toString().startsWith(TEST_NAME));
-				elements++;
-
-				if(format.getRecordsReadFromBlock() == recordsUntilCheckpoint) {
-
-					// do the whole checkpoint-restore procedure and see if we pick up from where we left off.
-					Tuple2<Long, Long> state = format.getCurrentState();
-
-					// this is to make sure that nothing stays from the previous format
-					// (as it is going to be in the normal case)
-					format = new AvroInputFormat<>(new Path(testFile.getAbsolutePath()), User.class);
-
-					format.reopen(splits[i], state);
-					assertEquals(format.getRecordsReadFromBlock(), recordsUntilCheckpoint);
-				}
-				elementsPerSplit[i]++;
-			}
-			format.close();
-		}
-
-		Assert.assertEquals(1539, elementsPerSplit[0]);
-		Assert.assertEquals(1026, elementsPerSplit[1]);
-		Assert.assertEquals(1539, elementsPerSplit[2]);
-		Assert.assertEquals(896, elementsPerSplit[3]);
-		Assert.assertEquals(NUM_RECORDS, elements);
-		format.close();
-	}
-
-	@Test
-	public void testAvroRecovery() throws Exception {
-		final int recordsUntilCheckpoint = 132;
-
-		Configuration parameters = new Configuration();
-
-		AvroInputFormat<User> format = new AvroInputFormat<User>(new Path(testFile.getAbsolutePath()), User.class);
-		format.configure(parameters);
-
-		FileInputSplit[] splits = format.createInputSplits(4);
-		assertEquals(splits.length, 4);
-
-		int elements = 0;
-		int elementsPerSplit[] = new int[4];
-		for(int i = 0; i < splits.length; i++) {
-			format.open(splits[i]);
-			while(!format.reachedEnd()) {
-				User u = format.nextRecord(null);
-				Assert.assertTrue(u.getName().toString().startsWith(TEST_NAME));
-				elements++;
-
-				if(format.getRecordsReadFromBlock() == recordsUntilCheckpoint) {
-
-					// do the whole checkpoint-restore procedure and see if we pick up from where we left off.
-					Tuple2<Long, Long> state = format.getCurrentState();
-
-					// this is to make sure that nothing stays from the previous format
-					// (as it is going to be in the normal case)
-					format = new AvroInputFormat<>(new Path(testFile.getAbsolutePath()), User.class);
-
-					format.reopen(splits[i], state);
-					assertEquals(format.getRecordsReadFromBlock(), recordsUntilCheckpoint);
-				}
-				elementsPerSplit[i]++;
-			}
-			format.close();
-		}
-
-		Assert.assertEquals(1539, elementsPerSplit[0]);
-		Assert.assertEquals(1026, elementsPerSplit[1]);
-		Assert.assertEquals(1539, elementsPerSplit[2]);
-		Assert.assertEquals(896, elementsPerSplit[3]);
-		Assert.assertEquals(NUM_RECORDS, elements);
-		format.close();
-	}
-
-	/*
-	This test is gave the reference values for the test of Flink's IF.
-
-	This dependency needs to be added
-
-        <dependency>
-            <groupId>org.apache.avro</groupId>
-            <artifactId>avro-mapred</artifactId>
-            <version>1.7.6</version>
-        </dependency>
-
-	@Test
-	public void testHadoop() throws Exception {
-		JobConf jf = new JobConf();
-		FileInputFormat.addInputPath(jf, new org.apache.hadoop.fs.Path(testFile.toURI()));
-		jf.setBoolean(org.apache.avro.mapred.AvroInputFormat.IGNORE_FILES_WITHOUT_EXTENSION_KEY, false);
-		org.apache.avro.mapred.AvroInputFormat<User> format = new org.apache.avro.mapred.AvroInputFormat<User>();
-		InputSplit[] sp = format.getSplits(jf, 4);
-		int elementsPerSplit[] = new int[4];
-		int cnt = 0;
-		int i = 0;
-		for(InputSplit s:sp) {
-			RecordReader<AvroWrapper<User>, NullWritable> r = format.getRecordReader(s, jf, new HadoopDummyReporter());
-			AvroWrapper<User> k = r.createKey();
-			NullWritable v = r.createValue();
-
-			while(r.next(k,v)) {
-				cnt++;
-				elementsPerSplit[i]++;
-			}
-			i++;
-		}
-		System.out.println("Status "+Arrays.toString(elementsPerSplit));
-	} **/
-	
-	@After
-	public void deleteFiles() {
-		testFile.delete();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/AvroTypeExample.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/AvroTypeExample.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/AvroTypeExample.java
deleted file mode 100644
index 5a21691..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/AvroTypeExample.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.io.avro.example;
-
-import java.io.IOException;
-import java.util.Random;
-
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.io.GenericInputFormat;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-
-@SuppressWarnings("serial")
-public class AvroTypeExample {
-	
-	
-	public static void main(String[] args) throws Exception {
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<User> users = env.createInput(new UserGeneratingInputFormat());
-
-		users
-			.map(new NumberExtractingMapper())
-			.groupBy(1)
-			.reduceGroup(new ConcatenatingReducer())
-			.print();
-	}
-	
-	
-	public static final class NumberExtractingMapper implements MapFunction<User, Tuple2<User, Integer>> {
-		
-		@Override
-		public Tuple2<User, Integer> map(User user) {
-			return new Tuple2<User, Integer>(user, user.getFavoriteNumber());
-		}
-	}
-	
-	
-	public static final class ConcatenatingReducer implements GroupReduceFunction<Tuple2<User, Integer>, Tuple2<Integer, String>> {
-
-		@Override
-		public void reduce(Iterable<Tuple2<User, Integer>> values, Collector<Tuple2<Integer, String>> out) throws Exception {
-			int number = 0;
-			StringBuilder colors = new StringBuilder();
-			
-			for (Tuple2<User, Integer> u : values) {
-				number = u.f1;
-				colors.append(u.f0.getFavoriteColor()).append(" - ");
-			}
-			
-			colors.setLength(colors.length() - 3);
-			out.collect(new Tuple2<Integer, String>(number, colors.toString()));
-		}
-	}
-	
-	
-	public static final class UserGeneratingInputFormat extends GenericInputFormat<User> {
-
-		private static final long serialVersionUID = 1L;
-		
-		private static final int NUM = 100;
-		
-		private final Random rnd = new Random(32498562304986L);
-		
-		private static final String[] NAMES = { "Peter", "Bob", "Liddy", "Alexander", "Stan" };
-		
-		private static final String[] COLORS = { "mauve", "crimson", "copper", "sky", "grass" };
-		
-		private int count;
-		
-
-		@Override
-		public boolean reachedEnd() throws IOException {
-			return count >= NUM;
-		}
-
-		@Override
-		public User nextRecord(User reuse) throws IOException {
-			count++;
-			
-			User u = new User();
-			u.setName(NAMES[rnd.nextInt(NAMES.length)]);
-			u.setFavoriteColor(COLORS[rnd.nextInt(COLORS.length)]);
-			u.setFavoriteNumber(rnd.nextInt(87));
-			return u;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/User.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/User.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/User.java
deleted file mode 100644
index 4608f96..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/example/User.java
+++ /dev/null
@@ -1,269 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-/**
- * Autogenerated by Avro
- * 
- * DO NOT EDIT DIRECTLY
- */
-package org.apache.flink.api.io.avro.example;  
-@SuppressWarnings("all")
-@org.apache.avro.specific.AvroGenerated
-public class User extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
-  public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"User\",\"namespace\":\"org.apache.flink.api.avro.example\",\"fields\":[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"favorite_number\",\"type\":[\"int\",\"null\"]},{\"name\":\"favorite_color\",\"type\":[\"string\",\"null\"]}]}");
-  public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; }
-  @Deprecated public java.lang.CharSequence name;
-  @Deprecated public java.lang.Integer favorite_number;
-  @Deprecated public java.lang.CharSequence favorite_color;
-
-  /**
-   * Default constructor.  Note that this does not initialize fields
-   * to their default values from the schema.  If that is desired then
-   * one should use {@link #newBuilder()}. 
-   */
-  public User() {}
-
-  /**
-   * All-args constructor.
-   */
-  public User(java.lang.CharSequence name, java.lang.Integer favorite_number, java.lang.CharSequence favorite_color) {
-    this.name = name;
-    this.favorite_number = favorite_number;
-    this.favorite_color = favorite_color;
-  }
-
-  public org.apache.avro.Schema getSchema() { return SCHEMA$; }
-  // Used by DatumWriter.  Applications should not call. 
-  public java.lang.Object get(int field$) {
-    switch (field$) {
-    case 0: return name;
-    case 1: return favorite_number;
-    case 2: return favorite_color;
-    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
-    }
-  }
-  // Used by DatumReader.  Applications should not call. 
-  @SuppressWarnings(value="unchecked")
-  public void put(int field$, java.lang.Object value$) {
-    switch (field$) {
-    case 0: name = (java.lang.CharSequence)value$; break;
-    case 1: favorite_number = (java.lang.Integer)value$; break;
-    case 2: favorite_color = (java.lang.CharSequence)value$; break;
-    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
-    }
-  }
-
-  /**
-   * Gets the value of the 'name' field.
-   */
-  public java.lang.CharSequence getName() {
-    return name;
-  }
-
-  /**
-   * Sets the value of the 'name' field.
-   * @param value the value to set.
-   */
-  public void setName(java.lang.CharSequence value) {
-    this.name = value;
-  }
-
-  /**
-   * Gets the value of the 'favorite_number' field.
-   */
-  public java.lang.Integer getFavoriteNumber() {
-    return favorite_number;
-  }
-
-  /**
-   * Sets the value of the 'favorite_number' field.
-   * @param value the value to set.
-   */
-  public void setFavoriteNumber(java.lang.Integer value) {
-    this.favorite_number = value;
-  }
-
-  /**
-   * Gets the value of the 'favorite_color' field.
-   */
-  public java.lang.CharSequence getFavoriteColor() {
-    return favorite_color;
-  }
-
-  /**
-   * Sets the value of the 'favorite_color' field.
-   * @param value the value to set.
-   */
-  public void setFavoriteColor(java.lang.CharSequence value) {
-    this.favorite_color = value;
-  }
-
-  /** Creates a new User RecordBuilder */
-  public static org.apache.flink.api.io.avro.example.User.Builder newBuilder() {
-    return new org.apache.flink.api.io.avro.example.User.Builder();
-  }
-  
-  /** Creates a new User RecordBuilder by copying an existing Builder */
-  public static org.apache.flink.api.io.avro.example.User.Builder newBuilder(org.apache.flink.api.io.avro.example.User.Builder other) {
-    return new org.apache.flink.api.io.avro.example.User.Builder(other);
-  }
-  
-  /** Creates a new User RecordBuilder by copying an existing User instance */
-  public static org.apache.flink.api.io.avro.example.User.Builder newBuilder(org.apache.flink.api.io.avro.example.User other) {
-    return new org.apache.flink.api.io.avro.example.User.Builder(other);
-  }
-  
-  /**
-   * RecordBuilder for User instances.
-   */
-  public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase<User>
-    implements org.apache.avro.data.RecordBuilder<User> {
-
-    private java.lang.CharSequence name;
-    private java.lang.Integer favorite_number;
-    private java.lang.CharSequence favorite_color;
-
-    /** Creates a new Builder */
-    private Builder() {
-      super(org.apache.flink.api.io.avro.example.User.SCHEMA$);
-    }
-    
-    /** Creates a Builder by copying an existing Builder */
-    private Builder(org.apache.flink.api.io.avro.example.User.Builder other) {
-      super(other);
-      if (isValidValue(fields()[0], other.name)) {
-        this.name = data().deepCopy(fields()[0].schema(), other.name);
-        fieldSetFlags()[0] = true;
-      }
-      if (isValidValue(fields()[1], other.favorite_number)) {
-        this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number);
-        fieldSetFlags()[1] = true;
-      }
-      if (isValidValue(fields()[2], other.favorite_color)) {
-        this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color);
-        fieldSetFlags()[2] = true;
-      }
-    }
-    
-    /** Creates a Builder by copying an existing User instance */
-    private Builder(org.apache.flink.api.io.avro.example.User other) {
-            super(org.apache.flink.api.io.avro.example.User.SCHEMA$);
-      if (isValidValue(fields()[0], other.name)) {
-        this.name = data().deepCopy(fields()[0].schema(), other.name);
-        fieldSetFlags()[0] = true;
-      }
-      if (isValidValue(fields()[1], other.favorite_number)) {
-        this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number);
-        fieldSetFlags()[1] = true;
-      }
-      if (isValidValue(fields()[2], other.favorite_color)) {
-        this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color);
-        fieldSetFlags()[2] = true;
-      }
-    }
-
-    /** Gets the value of the 'name' field */
-    public java.lang.CharSequence getName() {
-      return name;
-    }
-    
-    /** Sets the value of the 'name' field */
-    public org.apache.flink.api.io.avro.example.User.Builder setName(java.lang.CharSequence value) {
-      validate(fields()[0], value);
-      this.name = value;
-      fieldSetFlags()[0] = true;
-      return this; 
-    }
-    
-    /** Checks whether the 'name' field has been set */
-    public boolean hasName() {
-      return fieldSetFlags()[0];
-    }
-    
-    /** Clears the value of the 'name' field */
-    public org.apache.flink.api.io.avro.example.User.Builder clearName() {
-      name = null;
-      fieldSetFlags()[0] = false;
-      return this;
-    }
-
-    /** Gets the value of the 'favorite_number' field */
-    public java.lang.Integer getFavoriteNumber() {
-      return favorite_number;
-    }
-    
-    /** Sets the value of the 'favorite_number' field */
-    public org.apache.flink.api.io.avro.example.User.Builder setFavoriteNumber(java.lang.Integer value) {
-      validate(fields()[1], value);
-      this.favorite_number = value;
-      fieldSetFlags()[1] = true;
-      return this; 
-    }
-    
-    /** Checks whether the 'favorite_number' field has been set */
-    public boolean hasFavoriteNumber() {
-      return fieldSetFlags()[1];
-    }
-    
-    /** Clears the value of the 'favorite_number' field */
-    public org.apache.flink.api.io.avro.example.User.Builder clearFavoriteNumber() {
-      favorite_number = null;
-      fieldSetFlags()[1] = false;
-      return this;
-    }
-
-    /** Gets the value of the 'favorite_color' field */
-    public java.lang.CharSequence getFavoriteColor() {
-      return favorite_color;
-    }
-    
-    /** Sets the value of the 'favorite_color' field */
-    public org.apache.flink.api.io.avro.example.User.Builder setFavoriteColor(java.lang.CharSequence value) {
-      validate(fields()[2], value);
-      this.favorite_color = value;
-      fieldSetFlags()[2] = true;
-      return this; 
-    }
-    
-    /** Checks whether the 'favorite_color' field has been set */
-    public boolean hasFavoriteColor() {
-      return fieldSetFlags()[2];
-    }
-    
-    /** Clears the value of the 'favorite_color' field */
-    public org.apache.flink.api.io.avro.example.User.Builder clearFavoriteColor() {
-      favorite_color = null;
-      fieldSetFlags()[2] = false;
-      return this;
-    }
-
-    @Override
-    public User build() {
-      try {
-        User record = new User();
-        record.name = fieldSetFlags()[0] ? this.name : (java.lang.CharSequence) defaultValue(fields()[0]);
-        record.favorite_number = fieldSetFlags()[1] ? this.favorite_number : (java.lang.Integer) defaultValue(fields()[1]);
-        record.favorite_color = fieldSetFlags()[2] ? this.favorite_color : (java.lang.CharSequence) defaultValue(fields()[2]);
-        return record;
-      } catch (Exception e) {
-        throw new org.apache.avro.AvroRuntimeException(e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroInputFormatTypeExtractionTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroInputFormatTypeExtractionTest.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroInputFormatTypeExtractionTest.java
deleted file mode 100644
index e245026..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroInputFormatTypeExtractionTest.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.java.io;
-
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.typeutils.PojoTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.core.fs.Path;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class AvroInputFormatTypeExtractionTest {
-
-	@Test
-	public void testTypeExtraction() {
-		try {
-			InputFormat<MyAvroType, ?> format = new AvroInputFormat<MyAvroType>(new Path("file:///ignore/this/file"), MyAvroType.class);
-
-			TypeInformation<?> typeInfoDirect = TypeExtractor.getInputFormatTypes(format);
-
-			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			DataSet<MyAvroType> input = env.createInput(format);
-			TypeInformation<?> typeInfoDataSet = input.getType();
-
-
-			Assert.assertTrue(typeInfoDirect instanceof PojoTypeInfo);
-			Assert.assertTrue(typeInfoDataSet instanceof PojoTypeInfo);
-
-			Assert.assertEquals(MyAvroType.class, typeInfoDirect.getTypeClass());
-			Assert.assertEquals(MyAvroType.class, typeInfoDataSet.getTypeClass());
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	public static final class MyAvroType {
-
-		public String theString;
-
-		public MyAvroType recursive;
-
-		private double aDouble;
-
-		public double getaDouble() {
-			return aDouble;
-		}
-
-		public void setaDouble(double aDouble) {
-			this.aDouble = aDouble;
-		}
-
-		public void setTheString(String theString) {
-			this.theString = theString;
-		}
-
-		public String getTheString() {
-			return theString;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java
deleted file mode 100644
index 4d6c6b7..0000000
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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 static org.apache.flink.api.java.io.AvroOutputFormat.Codec;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-
-import org.apache.avro.Schema;
-import org.apache.flink.api.io.avro.example.User;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
-
-/**
- * Tests for {@link AvroOutputFormat}
- */
-public class AvroOutputFormatTest {
-
-	@Test
-	public void testSetCodec() throws Exception {
-		// given
-		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
-
-		// when
-		try {
-			outputFormat.setCodec(Codec.SNAPPY);
-		} catch (Exception ex) {
-			// then
-			fail("unexpected exception");
-		}
-	}
-
-	@Test
-	public void testSetCodecError() throws Exception {
-		// given
-		boolean error = false;
-		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
-
-		// when
-		try {
-			outputFormat.setCodec(null);
-		} catch (Exception ex) {
-			error = true;
-		}
-
-		// then
-		assertTrue(error);
-	}
-
-	@Test
-	public void testSerialization() throws Exception {
-
-		serializeAndDeserialize(null, null);
-		serializeAndDeserialize(null, User.SCHEMA$);
-		for (final Codec codec : Codec.values()) {
-			serializeAndDeserialize(codec, null);
-			serializeAndDeserialize(codec, User.SCHEMA$);
-		}
-	}
-
-	private void serializeAndDeserialize(final Codec codec, final Schema schema) throws IOException, ClassNotFoundException {
-		// given
-		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
-		if (codec != null) {
-			outputFormat.setCodec(codec);
-		}
-		if (schema != null) {
-			outputFormat.setSchema(schema);
-		}
-
-		final ByteArrayOutputStream bos = new ByteArrayOutputStream();
-
-		// when
-		try (final ObjectOutputStream oos = new ObjectOutputStream(bos)) {
-			oos.writeObject(outputFormat);
-		}
-		try (final ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(bos.toByteArray()))) {
-			// then
-			Object o = ois.readObject();
-			assertTrue(o instanceof AvroOutputFormat);
-			final AvroOutputFormat<User> restored = (AvroOutputFormat<User>) o;
-			final Codec restoredCodec = (Codec) Whitebox.getInternalState(restored, "codec");
-			final Schema restoredSchema = (Schema) Whitebox.getInternalState(restored, "userDefinedSchema");
-
-			assertTrue(codec != null ? restoredCodec == codec : restoredCodec == null);
-			assertTrue(schema != null ? restoredSchema.equals(schema) : restoredSchema == null);
-		}
-	}
-
-	@Test
-	public void testCompression() throws Exception {
-		// given
-		final Path outputPath = new Path(File.createTempFile("avro-output-file","avro").getAbsolutePath());
-		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(outputPath,User.class);
-		outputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
-
-		final Path compressedOutputPath = new Path(File.createTempFile("avro-output-file","compressed.avro").getAbsolutePath());
-		final AvroOutputFormat<User> compressedOutputFormat = new AvroOutputFormat<>(compressedOutputPath,User.class);
-		compressedOutputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
-		compressedOutputFormat.setCodec(Codec.SNAPPY);
-
-		// when
-		output(outputFormat);
-		output(compressedOutputFormat);
-
-		// then
-		assertTrue(fileSize(outputPath) > fileSize(compressedOutputPath));
-
-		// cleanup
-		Files.delete(Paths.get(outputPath.getPath()));
-		Files.delete(Paths.get(compressedOutputPath.getPath()));
-	}
-
-	private long fileSize(Path path) throws IOException {
-		return Files.size(Paths.get(path.getPath()));
-	}
-
-	private void output(final AvroOutputFormat<User> outputFormat) throws IOException {
-		outputFormat.configure(new Configuration());
-		outputFormat.open(1,1);
-		for (int i = 0; i < 100; i++) {
-			outputFormat.writeRecord(new User("testUser",1,"blue"));
-		}
-		outputFormat.close();
-	}
-}


[32/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
new file mode 100644
index 0000000..c68fe28
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * 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 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;
+	private final int cachedHash;
+
+	public KafkaTopicPartition(String topic, int partition) {
+		this.topic = requireNonNull(topic);
+		this.partition = partition;
+		this.cachedHash = 31 * topic.hashCode() + partition;
+	}
+
+	// ------------------------------------------------------------------------
+	
+	public String getTopic() {
+		return topic;
+	}
+
+	public int getPartition() {
+		return partition;
+	}
+
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return "KafkaTopicPartition{" +
+				"topic='" + topic + '\'' +
+				", partition=" + partition +
+				'}';
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		else if (o instanceof KafkaTopicPartition) {
+			KafkaTopicPartition that = (KafkaTopicPartition) o;
+			return this.partition == that.partition && this.topic.equals(that.topic);
+		}
+		else {
+			return false;
+		}
+	}
+
+	@Override
+	public int hashCode() {
+		return cachedHash;
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	public static String toString(Map<KafkaTopicPartition, Long> map) {
+		StringBuilder sb = new StringBuilder();
+		for (Map.Entry<KafkaTopicPartition, Long> p: map.entrySet()) {
+			KafkaTopicPartition ktp = p.getKey();
+			sb.append(ktp.getTopic()).append(":").append(ktp.getPartition()).append("=").append(p.getValue()).append(", ");
+		}
+		return sb.toString();
+	}
+
+	public static String toString(List<KafkaTopicPartition> partitions) {
+		StringBuilder sb = new StringBuilder();
+		for (KafkaTopicPartition p: partitions) {
+			sb.append(p.getTopic()).append(":").append(p.getPartition()).append(", ");
+		}
+		return sb.toString();
+	}
+
+
+	public static List<KafkaTopicPartition> dropLeaderData(List<KafkaTopicPartitionLeader> partitionInfos) {
+		List<KafkaTopicPartition> ret = new ArrayList<>(partitionInfos.size());
+		for(KafkaTopicPartitionLeader ktpl: partitionInfos) {
+			ret.add(ktpl.getTopicPartition());
+		}
+		return ret;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
new file mode 100644
index 0000000..1959a05
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import org.apache.kafka.common.Node;
+
+import java.io.Serializable;
+
+/**
+ * Serializable Topic Partition info with leader Node information.
+ * This class is used at runtime.
+ */
+public class KafkaTopicPartitionLeader implements Serializable {
+
+	private static final long serialVersionUID = 9145855900303748582L;
+
+	private final int leaderId;
+	private final int leaderPort;
+	private final String leaderHost;
+	private final KafkaTopicPartition topicPartition;
+	private final int cachedHash;
+
+	public KafkaTopicPartitionLeader(KafkaTopicPartition topicPartition, Node leader) {
+		this.topicPartition = topicPartition;
+		if (leader == null) {
+			this.leaderId = -1;
+			this.leaderHost = null;
+			this.leaderPort = -1;
+		} else {
+			this.leaderId = leader.id();
+			this.leaderPort = leader.port();
+			this.leaderHost = leader.host();
+		}
+		int cachedHash = (leader == null) ? 14 : leader.hashCode();
+		this.cachedHash = 31 * cachedHash + topicPartition.hashCode();
+	}
+
+	public KafkaTopicPartition getTopicPartition() {
+		return topicPartition;
+	}
+
+	public Node getLeader() {
+		if (this.leaderId == -1) {
+			return null;
+		} else {
+			return new Node(leaderId, leaderHost, leaderPort);
+		}
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (!(o instanceof KafkaTopicPartitionLeader)) {
+			return false;
+		}
+
+		KafkaTopicPartitionLeader that = (KafkaTopicPartitionLeader) o;
+
+		if (!topicPartition.equals(that.topicPartition)) {
+			return false;
+		}
+		return leaderId == that.leaderId && leaderPort == that.leaderPort && leaderHost.equals(that.leaderHost);
+	}
+
+	@Override
+	public int hashCode() {
+		return cachedHash;
+	}
+
+	@Override
+	public String toString() {
+		return "KafkaTopicPartitionLeader{" +
+				"leaderId=" + leaderId +
+				", leaderPort=" + leaderPort +
+				", leaderHost='" + leaderHost + '\'' +
+				", topic=" + topicPartition.getTopic() +
+				", partition=" + topicPartition.getPartition() +
+				'}';
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java
new file mode 100644
index 0000000..7cb5f46
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.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;
+
+	/** The offset of the Kafka partition that has been committed */
+	private volatile long committedOffset;
+
+	// ------------------------------------------------------------------------
+	
+	public KafkaTopicPartitionState(KafkaTopicPartition partition, KPH kafkaPartitionHandle) {
+		this.partition = partition;
+		this.kafkaPartitionHandle = kafkaPartitionHandle;
+		this.offset = OFFSET_NOT_SET;
+		this.committedOffset = 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;
+	}
+
+	public final void setCommittedOffset(long offset) {
+		this.committedOffset = offset;
+	}
+
+	public final long getCommittedOffset() {
+		return committedOffset;
+	}
+
+	
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return "Partition: " + partition + ", KafkaPartitionHandle=" + kafkaPartitionHandle
+				+ ", offset=" + (isOffsetDefined() ? String.valueOf(offset) : "(not set)");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java
new file mode 100644
index 0000000..efdc73f
--- /dev/null
+++ b/flink-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, long kafkaEventTimestamp) {
+		return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp);
+	}
+	
+	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/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java
new file mode 100644
index 0000000..edf40ce
--- /dev/null
+++ b/flink-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, long kafkaEventTimestamp) {
+		return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp);
+	}
+
+	@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/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TypeUtil.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TypeUtil.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TypeUtil.java
new file mode 100644
index 0000000..7a41ade
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TypeUtil.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+public class TypeUtil {
+	private TypeUtil() {}
+
+	/**
+	 * Creates TypeInformation array for an array of Classes.
+	 * @param fieldTypes classes to extract type information from
+	 * @return type information
+	 */
+	public static TypeInformation<?>[] toTypeInfo(Class<?>[] fieldTypes) {
+		TypeInformation<?>[] typeInfos = new TypeInformation[fieldTypes.length];
+		for (int i = 0; i < fieldTypes.length; i++) {
+			typeInfos[i] = TypeExtractor.getForClass(fieldTypes[i]);
+		}
+		return typeInfos;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java
new file mode 100644
index 0000000..cedb696
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals.metrics;
+
+import org.apache.flink.metrics.Gauge;
+
+/**
+ * Gauge for getting the current value of a Kafka metric.
+ */
+public class KafkaMetricWrapper implements Gauge<Double> {
+	private final org.apache.kafka.common.Metric kafkaMetric;
+
+	public KafkaMetricWrapper(org.apache.kafka.common.Metric metric) {
+		this.kafkaMetric = metric;
+	}
+
+	@Override
+	public Double getValue() {
+		return kafkaMetric.value();
+	}
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
new file mode 100644
index 0000000..d170058
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util.serialization;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import java.io.IOException;
+
+
+/**
+ * DeserializationSchema that deserializes a JSON String into an ObjectNode.
+ * <p>
+ * Fields can be accessed by calling objectNode.get(&lt;name>).as(&lt;type>)
+ */
+public class JSONDeserializationSchema extends AbstractDeserializationSchema<ObjectNode> {
+	private ObjectMapper mapper;
+
+	@Override
+	public ObjectNode deserialize(byte[] message) throws IOException {
+		if (mapper == null) {
+			mapper = new ObjectMapper();
+		}
+		return mapper.readValue(message, ObjectNode.class);
+	}
+
+	@Override
+	public boolean isEndOfStream(ObjectNode nextElement) {
+		return false;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java
new file mode 100644
index 0000000..261a111
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util.serialization;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import java.io.IOException;
+
+import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass;
+
+/**
+ * DeserializationSchema that deserializes a JSON String into an ObjectNode.
+ * <p>
+ * Key fields can be accessed by calling objectNode.get("key").get(&lt;name>).as(&lt;type>)
+ * <p>
+ * Value fields can be accessed by calling objectNode.get("value").get(&lt;name>).as(&lt;type>)
+ * <p>
+ * Metadata fields can be accessed by calling objectNode.get("metadata").get(&lt;name>).as(&lt;type>) and include
+ * the "offset" (long), "topic" (String) and "partition" (int).
+ */
+public class JSONKeyValueDeserializationSchema implements KeyedDeserializationSchema<ObjectNode> {
+	private final boolean includeMetadata;
+	private ObjectMapper mapper;
+
+	public JSONKeyValueDeserializationSchema(boolean includeMetadata) {
+		this.includeMetadata = includeMetadata;
+	}
+
+	@Override
+	public ObjectNode deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
+		if (mapper == null) {
+			mapper = new ObjectMapper();
+		}
+		ObjectNode node = mapper.createObjectNode();
+		node.set("key", mapper.readValue(messageKey, JsonNode.class));
+		node.set("value", mapper.readValue(message, JsonNode.class));
+		if (includeMetadata) {
+			node.putObject("metadata")
+				.put("offset", offset)
+				.put("topic", topic)
+				.put("partition", partition);
+		}
+		return node;
+	}
+
+	@Override
+	public boolean isEndOfStream(ObjectNode nextElement) {
+		return false;
+	}
+
+	@Override
+	public TypeInformation<ObjectNode> getProducedType() {
+		return getForClass(ObjectNode.class);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
new file mode 100644
index 0000000..4344810
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.util.serialization;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.typeutils.RowTypeInfo;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * Deserialization schema from JSON to {@link Row}.
+ *
+ * <p>Deserializes the <code>byte[]</code> messages as a JSON object and reads
+ * the specified fields.
+ *
+ * <p>Failure during deserialization are forwarded as wrapped IOExceptions.
+ */
+public class JsonRowDeserializationSchema implements DeserializationSchema<Row> {
+
+	/** Field names to parse. Indices match fieldTypes indices. */
+	private final String[] fieldNames;
+
+	/** Types to parse fields as. Indices match fieldNames indices. */
+	private final TypeInformation<?>[] fieldTypes;
+
+	/** Object mapper for parsing the JSON. */
+	private final ObjectMapper objectMapper = new ObjectMapper();
+
+	/** Flag indicating whether to fail on a missing field. */
+	private boolean failOnMissingField;
+
+	/**
+	 * Creates a JSON deserialization schema for the given fields and type classes.
+	 *
+	 * @param fieldNames Names of JSON fields to parse.
+	 * @param fieldTypes Type classes to parse JSON fields as.
+	 */
+	public JsonRowDeserializationSchema(String[] fieldNames, Class<?>[] fieldTypes) {
+		this.fieldNames = Preconditions.checkNotNull(fieldNames, "Field names");
+
+		this.fieldTypes = new TypeInformation[fieldTypes.length];
+		for (int i = 0; i < fieldTypes.length; i++) {
+			this.fieldTypes[i] = TypeExtractor.getForClass(fieldTypes[i]);
+		}
+
+		Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
+				"Number of provided field names and types does not match.");
+	}
+
+	/**
+	 * Creates a JSON deserialization schema for the given fields and types.
+	 *
+	 * @param fieldNames Names of JSON fields to parse.
+	 * @param fieldTypes Types to parse JSON fields as.
+	 */
+	public JsonRowDeserializationSchema(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
+		this.fieldNames = Preconditions.checkNotNull(fieldNames, "Field names");
+		this.fieldTypes = Preconditions.checkNotNull(fieldTypes, "Field types");
+
+		Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
+				"Number of provided field names and types does not match.");
+	}
+
+	@Override
+	public Row deserialize(byte[] message) throws IOException {
+		try {
+			JsonNode root = objectMapper.readTree(message);
+
+			Row row = new Row(fieldNames.length);
+			for (int i = 0; i < fieldNames.length; i++) {
+				JsonNode node = root.get(fieldNames[i]);
+
+				if (node == null) {
+					if (failOnMissingField) {
+						throw new IllegalStateException("Failed to find field with name '"
+								+ fieldNames[i] + "'.");
+					} else {
+						row.setField(i, null);
+					}
+				} else {
+					// Read the value as specified type
+					Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass());
+					row.setField(i, value);
+				}
+			}
+
+			return row;
+		} catch (Throwable t) {
+			throw new IOException("Failed to deserialize JSON object.", t);
+		}
+	}
+
+	@Override
+	public boolean isEndOfStream(Row nextElement) {
+		return false;
+	}
+
+	@Override
+	public TypeInformation<Row> getProducedType() {
+		return new RowTypeInfo(fieldTypes);
+	}
+
+	/**
+	 * Configures the failure behaviour if a JSON field is missing.
+	 *
+	 * <p>By default, a missing field is ignored and the field is set to null.
+	 *
+	 * @param failOnMissingField Flag indicating whether to fail or not on a missing field.
+	 */
+	public void setFailOnMissingField(boolean failOnMissingField) {
+		this.failOnMissingField = failOnMissingField;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
new file mode 100644
index 0000000..077ff13
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util.serialization;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.util.Preconditions;
+
+
+/**
+ * Serialization schema that serializes an object into a JSON bytes.
+ *
+ * <p>Serializes the input {@link Row} object into a JSON string and
+ * converts it into <code>byte[]</code>.
+ *
+ * <p>Result <code>byte[]</code> messages can be deserialized using
+ * {@link JsonRowDeserializationSchema}.
+ */
+public class JsonRowSerializationSchema implements SerializationSchema<Row> {
+	/** Fields names in the input Row object */
+	private final String[] fieldNames;
+	/** Object mapper that is used to create output JSON objects */
+	private static ObjectMapper mapper = new ObjectMapper();
+
+	/**
+	 * Creates a JSON serialization schema for the given fields and types.
+	 *
+	 * @param fieldNames Names of JSON fields to parse.
+	 */
+	public JsonRowSerializationSchema(String[] fieldNames) {
+		this.fieldNames = Preconditions.checkNotNull(fieldNames);
+	}
+
+	@Override
+	public byte[] serialize(Row row) {
+		if (row.productArity() != fieldNames.length) {
+			throw new IllegalStateException(String.format(
+				"Number of elements in the row %s is different from number of field names: %d", row, fieldNames.length));
+		}
+
+		ObjectNode objectNode = mapper.createObjectNode();
+
+		for (int i = 0; i < row.productArity(); i++) {
+			JsonNode node = mapper.valueToTree(row.productElement(i));
+			objectNode.set(fieldNames[i], node);
+		}
+
+		try {
+			return mapper.writeValueAsBytes(objectNode);
+		} catch (Exception e) {
+			throw new RuntimeException("Failed to serialize row", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java
new file mode 100644
index 0000000..01e72ca
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.util.serialization;
+
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * The deserialization schema describes how to turn the byte key / value messages delivered by certain
+ * data sources (for example Apache Kafka) into data types (Java/Scala objects) that are
+ * processed by Flink.
+ * 
+ * @param <T> The type created by the keyed deserialization schema.
+ */
+public interface KeyedDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
+
+	/**
+	 * Deserializes the byte message.
+	 *
+	 * @param messageKey the key as a byte array (null if no key has been set)
+	 * @param message The message, as a byte array. (null if the message was empty or deleted)
+	 * @param partition The partition the message has originated from
+	 * @param offset the offset of the message in the original source (for example the Kafka offset)  @return The deserialized message as an object.
+	 */
+	T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException;
+
+	/**
+	 * Method to decide whether the element signals the end of the stream. If
+	 * true is returned the element won't be emitted.
+	 * 
+	 * @param nextElement The element to test for the end-of-stream signal.
+	 * @return True, if the element signals end of stream, false otherwise.
+	 */
+	boolean isEndOfStream(T nextElement);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
new file mode 100644
index 0000000..4b9dba2
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util.serialization;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import java.io.IOException;
+
+/**
+ * A simple wrapper for using the DeserializationSchema with the KeyedDeserializationSchema
+ * interface
+ * @param <T> The type created by the deserialization schema.
+ */
+public class KeyedDeserializationSchemaWrapper<T> implements KeyedDeserializationSchema<T> {
+
+	private static final long serialVersionUID = 2651665280744549932L;
+
+	private final DeserializationSchema<T> deserializationSchema;
+
+	public KeyedDeserializationSchemaWrapper(DeserializationSchema<T> deserializationSchema) {
+		this.deserializationSchema = deserializationSchema;
+	}
+	@Override
+	public T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
+		return deserializationSchema.deserialize(message);
+	}
+
+	@Override
+	public boolean isEndOfStream(T nextElement) {
+		return deserializationSchema.isEndOfStream(nextElement);
+	}
+
+	@Override
+	public TypeInformation<T> getProducedType() {
+		return deserializationSchema.getProducedType();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
new file mode 100644
index 0000000..701281e
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util.serialization;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema describes how to turn a data object into a different serialized
+ * representation. Most data sinks (for example Apache Kafka) require the data to be handed
+ * to them in a specific format (for example as byte strings).
+ * 
+ * @param <T> The type to be serialized.
+ */
+public interface KeyedSerializationSchema<T> extends Serializable {
+
+	/**
+	 * Serializes the key of the incoming element to a byte array
+	 * This method might return null if no key is available.
+	 *
+	 * @param element The incoming element to be serialized
+	 * @return the key of the element as a byte array
+	 */
+	byte[] serializeKey(T element);
+
+
+	/**
+	 * Serializes the value of the incoming element to a byte array
+	 * 
+	 * @param element The incoming element to be serialized
+	 * @return the value of the element as a byte array
+	 */
+	byte[] serializeValue(T element);
+
+	/**
+	 * Optional method to determine the target topic for the element
+	 *
+	 * @param element Incoming element to determine the target topic from
+	 * @return null or the target topic
+	 */
+	String getTargetTopic(T element);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
new file mode 100644
index 0000000..1b3e486
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util.serialization;
+
+/**
+ * A simple wrapper for using the SerializationSchema with the KeyedDeserializationSchema
+ * interface
+ * @param <T> The type to serialize
+ */
+public class KeyedSerializationSchemaWrapper<T> implements KeyedSerializationSchema<T> {
+
+	private static final long serialVersionUID = 1351665280744549933L;
+
+	private final SerializationSchema<T> serializationSchema;
+
+	public KeyedSerializationSchemaWrapper(SerializationSchema<T> serializationSchema) {
+		this.serializationSchema = serializationSchema;
+	}
+
+	@Override
+	public byte[] serializeKey(T element) {
+		return null;
+	}
+
+	@Override
+	public byte[] serializeValue(T element) {
+		return serializationSchema.serialize(element);
+	}
+
+	@Override
+	public String getTargetTopic(T element) {
+		return null; // we are never overriding the topic
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java
new file mode 100644
index 0000000..51bc8d1
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.util.serialization;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.runtime.util.DataInputDeserializer;
+import org.apache.flink.runtime.util.DataOutputSerializer;
+
+import java.io.IOException;
+
+/**
+ * A serialization and deserialization schema for Key Value Pairs that uses Flink's serialization stack to
+ * transform typed from and to byte arrays.
+ * 
+ * @param <K> The key type to be serialized.
+ * @param <V> The value type to be serialized.
+ */
+public class TypeInformationKeyValueSerializationSchema<K, V> implements KeyedDeserializationSchema<Tuple2<K, V>>, KeyedSerializationSchema<Tuple2<K,V>> {
+
+	private static final long serialVersionUID = -5359448468131559102L;
+
+	/** The serializer for the key */
+	private final TypeSerializer<K> keySerializer;
+
+	/** The serializer for the value */
+	private final TypeSerializer<V> valueSerializer;
+
+	/** reusable input deserialization buffer */
+	private final DataInputDeserializer inputDeserializer;
+	
+	/** reusable output serialization buffer for the key */
+	private transient DataOutputSerializer keyOutputSerializer;
+
+	/** reusable output serialization buffer for the value */
+	private transient DataOutputSerializer valueOutputSerializer;
+	
+	
+	/** The type information, to be returned by {@link #getProducedType()}. It is
+	 * transient, because it is not serializable. Note that this means that the type information
+	 * is not available at runtime, but only prior to the first serialization / deserialization */
+	private final transient TypeInformation<Tuple2<K, V>> typeInfo;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new de-/serialization schema for the given types.
+	 *
+	 * @param keyTypeInfo The type information for the key type de-/serialized by this schema.
+	 * @param valueTypeInfo The type information for the value type de-/serialized by this schema.
+	 * @param ec The execution config, which is used to parametrize the type serializers.
+	 */
+	public TypeInformationKeyValueSerializationSchema(TypeInformation<K> keyTypeInfo, TypeInformation<V> valueTypeInfo, ExecutionConfig ec) {
+		this.typeInfo = new TupleTypeInfo<>(keyTypeInfo, valueTypeInfo);
+		this.keySerializer = keyTypeInfo.createSerializer(ec);
+		this.valueSerializer = valueTypeInfo.createSerializer(ec);
+		this.inputDeserializer = new DataInputDeserializer();
+	}
+
+	/**
+	 * Creates a new de-/serialization schema for the given types. This constructor accepts the types
+	 * as classes and internally constructs the type information from the classes.
+	 * 
+	 * <p>If the types are parametrized and cannot be fully defined via classes, use the constructor
+	 * that accepts {@link TypeInformation} instead.
+	 * 
+	 * @param keyClass The class of the key de-/serialized by this schema.
+	 * @param valueClass The class of the value de-/serialized by this schema.
+	 * @param config The execution config, which is used to parametrize the type serializers.
+	 */
+	public TypeInformationKeyValueSerializationSchema(Class<K> keyClass, Class<V> valueClass, ExecutionConfig config) {
+		this(TypeExtractor.createTypeInfo(keyClass), TypeExtractor.createTypeInfo(valueClass), config);
+	}
+
+	// ------------------------------------------------------------------------
+
+
+	@Override
+	public Tuple2<K, V> deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
+		K key = null;
+		V value = null;
+		
+		if (messageKey != null) {
+			inputDeserializer.setBuffer(messageKey, 0, messageKey.length);
+			key = keySerializer.deserialize(inputDeserializer);
+		}
+		if (message != null) {
+			inputDeserializer.setBuffer(message, 0, message.length);
+			value = valueSerializer.deserialize(inputDeserializer);
+		}
+		return new Tuple2<>(key, value);
+	}
+
+	/**
+	 * This schema never considers an element to signal end-of-stream, so this method returns always false.
+	 * @param nextElement The element to test for the end-of-stream signal.
+	 * @return Returns false.
+	 */
+	@Override
+	public boolean isEndOfStream(Tuple2<K,V> nextElement) {
+		return false;
+	}
+
+
+	@Override
+	public byte[] serializeKey(Tuple2<K, V> element) {
+		if (element.f0 == null) {
+			return null;
+		} else {
+			// key is not null. serialize it:
+			if (keyOutputSerializer == null) {
+				keyOutputSerializer = new DataOutputSerializer(16);
+			}
+			try {
+				keySerializer.serialize(element.f0, keyOutputSerializer);
+			}
+			catch (IOException e) {
+				throw new RuntimeException("Unable to serialize record", e);
+			}
+			// check if key byte array size changed
+			byte[] res = keyOutputSerializer.getByteArray();
+			if (res.length != keyOutputSerializer.length()) {
+				byte[] n = new byte[keyOutputSerializer.length()];
+				System.arraycopy(res, 0, n, 0, keyOutputSerializer.length());
+				res = n;
+			}
+			keyOutputSerializer.clear();
+			return res;
+		}
+	}
+
+	@Override
+	public byte[] serializeValue(Tuple2<K, V> element) {
+		// if the value is null, its serialized value is null as well.
+		if (element.f1 == null) {
+			return null;
+		}
+
+		if (valueOutputSerializer == null) {
+			valueOutputSerializer = new DataOutputSerializer(16);
+		}
+
+		try {
+			valueSerializer.serialize(element.f1, valueOutputSerializer);
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Unable to serialize record", e);
+		}
+
+		byte[] res = valueOutputSerializer.getByteArray();
+		if (res.length != valueOutputSerializer.length()) {
+			byte[] n = new byte[valueOutputSerializer.length()];
+			System.arraycopy(res, 0, n, 0, valueOutputSerializer.length());
+			res = n;
+		}
+		valueOutputSerializer.clear();
+		return res;
+	}
+
+	@Override
+	public String getTargetTopic(Tuple2<K, V> element) {
+		return null; // we are never overriding the topic
+	}
+
+
+	@Override
+	public TypeInformation<Tuple2<K,V>> getProducedType() {
+		if (typeInfo != null) {
+			return typeInfo;
+		}
+		else {
+			throw new IllegalStateException(
+					"The type information is not available after this class has been serialized and distributed.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
new file mode 100644
index 0000000..b96ba30
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
+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;
+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.Assert;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class FlinkKafkaConsumerBaseTest {
+
+	/**
+	 * Tests that not both types of timestamp extractors / watermark generators can be used.
+	 */
+	@Test
+	public void testEitherWatermarkExtractor() {
+		try {
+			new DummyFlinkKafkaConsumer<>().assignTimestampsAndWatermarks((AssignerWithPeriodicWatermarks<Object>) null);
+			fail();
+		} catch (NullPointerException ignored) {}
+
+		try {
+			new DummyFlinkKafkaConsumer<>().assignTimestampsAndWatermarks((AssignerWithPunctuatedWatermarks<Object>) 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.assignTimestampsAndWatermarks(periodicAssigner);
+		try {
+			c1.assignTimestampsAndWatermarks(punctuatedAssigner);
+			fail();
+		} catch (IllegalStateException ignored) {}
+
+		DummyFlinkKafkaConsumer<String> c2 = new DummyFlinkKafkaConsumer<>();
+		c2.assignTimestampsAndWatermarks(punctuatedAssigner);
+		try {
+			c2.assignTimestampsAndWatermarks(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);
+		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
+		TestingListState<Tuple2<KafkaTopicPartition, Long>> listState = new TestingListState<>();
+		when(operatorStateStore.getOperatorState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState);
+
+		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(1, 1));
+
+		assertFalse(listState.get().iterator().hasNext());
+		consumer.notifyCheckpointComplete(66L);
+	}
+
+	/**
+	 * Tests that no checkpoints happen when the fetcher is not running.
+	 */
+	@Test
+	public void checkRestoredCheckpointWhenFetcherNotReady() throws Exception {
+		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
+
+		TestingListState<Serializable> listState = new TestingListState<>();
+		listState.add(Tuple2.of(new KafkaTopicPartition("abc", 13), 16768L));
+		listState.add(Tuple2.of(new KafkaTopicPartition("def", 7), 987654321L));
+
+		FlinkKafkaConsumerBase<String> consumer = getConsumer(null, new LinkedMap(), true);
+
+		when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(listState);
+
+		StateInitializationContext initializationContext = mock(StateInitializationContext.class);
+
+		when(initializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
+		when(initializationContext.isRestored()).thenReturn(true);
+
+		consumer.initializeState(initializationContext);
+
+		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(17, 17));
+
+		// ensure that the list was cleared and refilled. while this is an implementation detail, we use it here
+		// to figure out that snapshotState() actually did something.
+		Assert.assertTrue(listState.isClearCalled());
+
+		Set<Serializable> expected = new HashSet<>();
+
+		for (Serializable serializable : listState.get()) {
+			expected.add(serializable);
+		}
+
+		int counter = 0;
+
+		for (Serializable serializable : listState.get()) {
+			assertTrue(expected.contains(serializable));
+			counter++;
+		}
+
+		assertEquals(expected.size(), counter);
+	}
+
+	/**
+	 * 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);
+
+		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
+		TestingListState<Serializable> listState = new TestingListState<>();
+		when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(listState);
+
+		StateInitializationContext initializationContext = mock(StateInitializationContext.class);
+
+		when(initializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
+		when(initializationContext.isRestored()).thenReturn(false);
+
+		consumer.initializeState(initializationContext);
+
+		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(17, 17));
+
+		assertFalse(listState.get().iterator().hasNext());
+	}
+
+	/**
+	 * Tests that on snapshots, states and offsets to commit to Kafka are correct
+	 */
+	@Test
+	public void checkUseFetcherWhenNoCheckpoint() throws Exception {
+
+		FlinkKafkaConsumerBase<String> consumer = getConsumer(null, new LinkedMap(), true);
+		List<KafkaTopicPartition> partitionList = new ArrayList<>(1);
+		partitionList.add(new KafkaTopicPartition("test", 0));
+		consumer.setSubscribedPartitions(partitionList);
+
+		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
+		TestingListState<Serializable> listState = new TestingListState<>();
+		when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(listState);
+
+		StateInitializationContext initializationContext = mock(StateInitializationContext.class);
+
+		when(initializationContext.getOperatorStateStore()).thenReturn(operatorStateStore);
+
+		// make the context signal that there is no restored state, then validate that
+		when(initializationContext.isRestored()).thenReturn(false);
+		consumer.initializeState(initializationContext);
+		consumer.run(mock(SourceFunction.SourceContext.class));
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testSnapshotState() throws Exception {
+
+		// --------------------------------------------------------------------
+		//   prepare fake states
+		// --------------------------------------------------------------------
+
+		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<>();
+		state3.put(new KafkaTopicPartition("abc", 13), 16780L);
+		state3.put(new KafkaTopicPartition("def", 7), 987654377L);
+
+		// --------------------------------------------------------------------
+		
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+		when(fetcher.snapshotCurrentState()).thenReturn(state1, state2, state3);
+			
+		final LinkedMap pendingOffsetsToCommit = new LinkedMap();
+	
+		FlinkKafkaConsumerBase<String> consumer = getConsumer(fetcher, pendingOffsetsToCommit, true);
+		assertEquals(0, pendingOffsetsToCommit.size());
+
+		OperatorStateStore backend = mock(OperatorStateStore.class);
+
+		TestingListState<Serializable> listState = new TestingListState<>();
+
+		when(backend.getSerializableListState(Matchers.any(String.class))).thenReturn(listState);
+
+		StateInitializationContext initializationContext = mock(StateInitializationContext.class);
+
+		when(initializationContext.getOperatorStateStore()).thenReturn(backend);
+		when(initializationContext.isRestored()).thenReturn(false, true, true, true);
+
+		consumer.initializeState(initializationContext);
+
+		// checkpoint 1
+		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(138, 138));
+
+		HashMap<KafkaTopicPartition, Long> snapshot1 = new HashMap<>();
+
+		for (Serializable serializable : listState.get()) {
+			Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 = (Tuple2<KafkaTopicPartition, Long>) serializable;
+			snapshot1.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
+		}
+
+		assertEquals(state1, snapshot1);
+		assertEquals(1, pendingOffsetsToCommit.size());
+		assertEquals(state1, pendingOffsetsToCommit.get(138L));
+
+		// checkpoint 2
+		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(140, 140));
+
+		HashMap<KafkaTopicPartition, Long> snapshot2 = new HashMap<>();
+
+		for (Serializable serializable : listState.get()) {
+			Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 = (Tuple2<KafkaTopicPartition, Long>) serializable;
+			snapshot2.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
+		}
+
+		assertEquals(state2, snapshot2);
+		assertEquals(2, pendingOffsetsToCommit.size());
+		assertEquals(state2, pendingOffsetsToCommit.get(140L));
+		
+		// ack checkpoint 1
+		consumer.notifyCheckpointComplete(138L);
+		assertEquals(1, pendingOffsetsToCommit.size());
+		assertTrue(pendingOffsetsToCommit.containsKey(140L));
+
+		// checkpoint 3
+		consumer.snapshotState(new StateSnapshotContextSynchronousImpl(141, 141));
+
+		HashMap<KafkaTopicPartition, Long> snapshot3 = new HashMap<>();
+
+		for (Serializable serializable : listState.get()) {
+			Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 = (Tuple2<KafkaTopicPartition, Long>) serializable;
+			snapshot3.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
+		}
+
+		assertEquals(state3, snapshot3);
+		assertEquals(2, pendingOffsetsToCommit.size());
+		assertEquals(state3, pendingOffsetsToCommit.get(141L));
+		
+		// ack checkpoint 3, subsumes number 2
+		consumer.notifyCheckpointComplete(141L);
+		assertEquals(0, pendingOffsetsToCommit.size());
+
+
+		consumer.notifyCheckpointComplete(666); // invalid checkpoint
+		assertEquals(0, pendingOffsetsToCommit.size());
+
+		OperatorStateStore operatorStateStore = mock(OperatorStateStore.class);
+		listState = new TestingListState<>();
+		when(operatorStateStore.getOperatorState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState);
+
+		// create 500 snapshots
+		for (int i = 100; i < 600; i++) {
+			consumer.snapshotState(new StateSnapshotContextSynchronousImpl(i, i));
+			listState.clear();
+		}
+		assertEquals(FlinkKafkaConsumerBase.MAX_NUM_PENDING_CHECKPOINTS, pendingOffsetsToCommit.size());
+
+		// commit only the second last
+		consumer.notifyCheckpointComplete(598);
+		assertEquals(1, pendingOffsetsToCommit.size());
+
+		// access invalid checkpoint
+		consumer.notifyCheckpointComplete(590);
+
+		// and the last
+		consumer.notifyCheckpointComplete(599);
+		assertEquals(0, pendingOffsetsToCommit.size());
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static <T> FlinkKafkaConsumerBase<T> getConsumer(
+			AbstractFetcher<T, ?> fetcher, LinkedMap pendingOffsetsToCommit, 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("pendingOffsetsToCommit");
+		mapField.setAccessible(true);
+		mapField.set(consumer, pendingOffsetsToCommit);
+
+		Field runningField = FlinkKafkaConsumerBase.class.getDeclaredField("running");
+		runningField.setAccessible(true);
+		runningField.set(consumer, running);
+
+		return consumer;
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
+		private static final long serialVersionUID = 1L;
+
+		@SuppressWarnings("unchecked")
+		public DummyFlinkKafkaConsumer() {
+			super(Arrays.asList("dummy-topic"), (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 {
+			AbstractFetcher<T, ?> fetcher = mock(AbstractFetcher.class);
+			doAnswer(new Answer() {
+				@Override
+				public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+					Assert.fail("Trying to restore offsets even though there was no restore state.");
+					return null;
+				}
+			}).when(fetcher).restoreOffsets(any(HashMap.class));
+			return fetcher;
+		}
+
+		@Override
+		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
+			return Collections.emptyList();
+		}
+
+		@Override
+		public RuntimeContext getRuntimeContext() {
+			return mock(StreamingRuntimeContext.class);
+		}
+	}
+
+	private static final class TestingListState<T> implements ListState<T> {
+
+		private final List<T> list = new ArrayList<>();
+		private boolean clearCalled = false;
+
+		@Override
+		public void clear() {
+			list.clear();
+			clearCalled = true;
+		}
+
+		@Override
+		public Iterable<T> get() throws Exception {
+			return list;
+		}
+
+		@Override
+		public void add(T value) throws Exception {
+			list.add(value);
+		}
+
+		public List<T> getList() {
+			return list;
+		}
+
+		public boolean isClearCalled() {
+			return clearCalled;
+		}
+	}
+}


[08/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java
deleted file mode 100644
index 25040eb..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java
+++ /dev/null
@@ -1,387 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internal;
-
-import org.apache.flink.streaming.connectors.kafka.internal.Handover.WakeupException;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Random;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-/**
- * Tests for the {@link Handover} between Kafka Consumer Thread and the fetcher's main thread. 
- */
-public class HandoverTest {
-
-	// ------------------------------------------------------------------------
-	//  test produce / consumer
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testWithVariableProducer() throws Exception {
-		runProducerConsumerTest(500, 2, 0);
-	}
-
-	@Test
-	public void testWithVariableConsumer() throws Exception {
-		runProducerConsumerTest(500, 0, 2);
-	}
-
-	@Test
-	public void testWithVariableBoth() throws Exception {
-		runProducerConsumerTest(500, 2, 2);
-	}
-
-	// ------------------------------------------------------------------------
-	//  test error propagation
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testPublishErrorOnEmptyHandover() throws Exception {
-		final Handover handover = new Handover();
-
-		Exception error = new Exception();
-		handover.reportError(error);
-
-		try {
-			handover.pollNext();
-			fail("should throw an exception");
-		}
-		catch (Exception e) {
-			assertEquals(error, e);
-		}
-	}
-
-	@Test
-	public void testPublishErrorOnFullHandover() throws Exception {
-		final Handover handover = new Handover();
-		handover.produce(createTestRecords());
-
-		IOException error = new IOException();
-		handover.reportError(error);
-
-		try {
-			handover.pollNext();
-			fail("should throw an exception");
-		}
-		catch (Exception e) {
-			assertEquals(error, e);
-		}
-	}
-
-	@Test
-	public void testExceptionMarksClosedOnEmpty() throws Exception {
-		final Handover handover = new Handover();
-
-		IllegalStateException error = new IllegalStateException();
-		handover.reportError(error);
-
-		try {
-			handover.produce(createTestRecords());
-			fail("should throw an exception");
-		}
-		catch (Handover.ClosedException e) {
-			// expected
-		}
-	}
-
-	@Test
-	public void testExceptionMarksClosedOnFull() throws Exception {
-		final Handover handover = new Handover();
-		handover.produce(createTestRecords());
-
-		LinkageError error = new LinkageError();
-		handover.reportError(error);
-
-		try {
-			handover.produce(createTestRecords());
-			fail("should throw an exception");
-		}
-		catch (Handover.ClosedException e) {
-			// expected
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  test closing behavior
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testCloseEmptyForConsumer() throws Exception {
-		final Handover handover = new Handover();
-		handover.close();
-
-		try {
-			handover.pollNext();
-			fail("should throw an exception");
-		}
-		catch (Handover.ClosedException e) {
-			// expected
-		}
-	}
-
-	@Test
-	public void testCloseFullForConsumer() throws Exception {
-		final Handover handover = new Handover();
-		handover.produce(createTestRecords());
-		handover.close();
-
-		try {
-			handover.pollNext();
-			fail("should throw an exception");
-		}
-		catch (Handover.ClosedException e) {
-			// expected
-		}
-	}
-
-	@Test
-	public void testCloseEmptyForProducer() throws Exception {
-		final Handover handover = new Handover();
-		handover.close();
-
-		try {
-			handover.produce(createTestRecords());
-			fail("should throw an exception");
-		}
-		catch (Handover.ClosedException e) {
-			// expected
-		}
-	}
-
-	@Test
-	public void testCloseFullForProducer() throws Exception {
-		final Handover handover = new Handover();
-		handover.produce(createTestRecords());
-		handover.close();
-
-		try {
-			handover.produce(createTestRecords());
-			fail("should throw an exception");
-		}
-		catch (Handover.ClosedException e) {
-			// expected
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  test wake up behavior
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testWakeupDoesNotWakeWhenEmpty() throws Exception {
-		Handover handover = new Handover();
-		handover.wakeupProducer();
-
-		// produce into a woken but empty handover
-		try {
-			handover.produce(createTestRecords());
-		}
-		catch (Handover.WakeupException e) {
-			fail();
-		}
-
-		// handover now has records, next time we wakeup and produce it needs
-		// to throw an exception
-		handover.wakeupProducer();
-		try {
-			handover.produce(createTestRecords());
-			fail("should throw an exception");
-		}
-		catch (Handover.WakeupException e) {
-			// expected
-		}
-
-		// empty the handover
-		assertNotNull(handover.pollNext());
-		
-		// producing into an empty handover should work
-		try {
-			handover.produce(createTestRecords());
-		}
-		catch (Handover.WakeupException e) {
-			fail();
-		}
-	}
-
-	@Test
-	public void testWakeupWakesOnlyOnce() throws Exception {
-		// create a full handover
-		final Handover handover = new Handover();
-		handover.produce(createTestRecords());
-
-		handover.wakeupProducer();
-
-		try {
-			handover.produce(createTestRecords());
-			fail();
-		} catch (WakeupException e) {
-			// expected
-		}
-
-		CheckedThread producer = new CheckedThread() {
-			@Override
-			public void go() throws Exception {
-				handover.produce(createTestRecords());
-			}
-		};
-		producer.start();
-
-		// the producer must go blocking
-		producer.waitUntilThreadHoldsLock(10000);
-
-		// release the thread by consuming something
-		assertNotNull(handover.pollNext());
-		producer.sync();
-	}
-
-	// ------------------------------------------------------------------------
-	//  utilities
-	// ------------------------------------------------------------------------
-
-	private void runProducerConsumerTest(int numRecords, int maxProducerDelay, int maxConsumerDelay) throws Exception {
-		// generate test data
-		@SuppressWarnings({"unchecked", "rawtypes"})
-		final ConsumerRecords<byte[], byte[]>[] data = new ConsumerRecords[numRecords];
-		for (int i = 0; i < numRecords; i++) {
-			data[i] = createTestRecords();
-		}
-
-		final Handover handover = new Handover();
-
-		ProducerThread producer = new ProducerThread(handover, data, maxProducerDelay);
-		ConsumerThread consumer = new ConsumerThread(handover, data, maxConsumerDelay);
-
-		consumer.start();
-		producer.start();
-
-		// sync first on the consumer, so it propagates assertion errors
-		consumer.sync();
-		producer.sync();
-	}
-
-	@SuppressWarnings("unchecked")
-	private static ConsumerRecords<byte[], byte[]> createTestRecords() {
-		return mock(ConsumerRecords.class);
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static abstract class CheckedThread extends Thread {
-
-		private volatile Throwable error;
-
-		public abstract void go() throws Exception;
-
-		@Override
-		public void run() {
-			try {
-				go();
-			}
-			catch (Throwable t) {
-				error = t;
-			}
-		}
-
-		public void sync() throws Exception {
-			join();
-			if (error != null) {
-				ExceptionUtils.rethrowException(error, error.getMessage());
-			}
-		}
-
-		public void waitUntilThreadHoldsLock(long timeoutMillis) throws InterruptedException, TimeoutException {
-			final long deadline = System.nanoTime() + timeoutMillis * 1_000_000;
-			
-			while (!isBlockedOrWaiting() && (System.nanoTime() < deadline)) {
-				Thread.sleep(1);
-			}
-
-			if (!isBlockedOrWaiting()) {
-				throw new TimeoutException();
-			}
-		}
-
-		private boolean isBlockedOrWaiting() {
-			State state = getState();
-			return state == State.BLOCKED || state == State.WAITING || state == State.TIMED_WAITING;
-		}
-	}
-
-	private static class ProducerThread extends CheckedThread {
-
-		private final Random rnd = new Random();
-		private final Handover handover;
-		private final ConsumerRecords<byte[], byte[]>[] data;
-		private final int maxDelay;
-
-		private ProducerThread(Handover handover, ConsumerRecords<byte[], byte[]>[] data, int maxDelay) {
-			this.handover = handover;
-			this.data = data;
-			this.maxDelay = maxDelay;
-		}
-
-		@Override
-		public void go() throws Exception {
-			for (ConsumerRecords<byte[], byte[]> rec : data) {
-				handover.produce(rec);
-
-				if (maxDelay > 0) {
-					int delay = rnd.nextInt(maxDelay);
-					Thread.sleep(delay);
-				}
-			}
-		}
-	}
-
-	private static class ConsumerThread extends CheckedThread {
-
-		private final Random rnd = new Random();
-		private final Handover handover;
-		private final ConsumerRecords<byte[], byte[]>[] data;
-		private final int maxDelay;
-
-		private ConsumerThread(Handover handover, ConsumerRecords<byte[], byte[]>[] data, int maxDelay) {
-			this.handover = handover;
-			this.data = data;
-			this.maxDelay = maxDelay;
-		}
-
-		@Override
-		public void go() throws Exception {
-			for (ConsumerRecords<byte[], byte[]> rec : data) {
-				ConsumerRecords<byte[], byte[]> next = handover.pollNext();
-
-				assertEquals(rec, next);
-
-				if (maxDelay > 0) {
-					int delay = rnd.nextInt(maxDelay);
-					Thread.sleep(delay);
-				}
-			}
-		}
-	}
-}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml
deleted file mode 100644
index ef71bde..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml
+++ /dev/null
@@ -1,212 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-kafka-base_2.10</artifactId>
-	<name>flink-connector-kafka-base</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<kafka.version>0.8.2.2</kafka.version>
-	</properties>
-
-	<dependencies>
-
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-			<!-- Projects depending on this project,
-			won't depend on flink-table. -->
-			<optional>true</optional>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.kafka</groupId>
-			<artifactId>kafka_${scala.binary.version}</artifactId>
-			<version>${kafka.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>com.sun.jmx</groupId>
-					<artifactId>jmxri</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jdmk</groupId>
-					<artifactId>jmxtools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-simple</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>net.sf.jopt-simple</groupId>
-					<artifactId>jopt-simple</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.scala-lang</groupId>
-					<artifactId>scala-reflect</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.scala-lang</groupId>
-					<artifactId>scala-compiler</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.yammer.metrics</groupId>
-					<artifactId>metrics-annotation</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.xerial.snappy</groupId>
-					<artifactId>snappy-java</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<!-- test dependencies -->
-		
-		<!-- force using the latest zkclient -->
-		<dependency>
-			<groupId>com.101tec</groupId>
-			<artifactId>zkclient</artifactId>
-			<version>0.7</version>
-			<type>jar</type>
-			<scope>test</scope>
-		</dependency>
-
-
-		<dependency>
-			<groupId>org.apache.curator</groupId>
-			<artifactId>curator-test</artifactId>
-			<version>${curator.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-metrics-jmx</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-minikdc</artifactId>
-			<version>${minikdc.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-	</dependencies>
-
-	<dependencyManagement>
-		<dependencies>
-			<dependency>
-				<groupId>com.101tec</groupId>
-				<artifactId>zkclient</artifactId>
-				<version>0.7</version>
-			</dependency>
-		</dependencies>
-	</dependencyManagement>
-	
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-			<!--
-            https://issues.apache.org/jira/browse/DIRSHARED-134
-            Required to pull the Mini-KDC transitive dependency
-            -->
-			<plugin>
-				<groupId>org.apache.felix</groupId>
-				<artifactId>maven-bundle-plugin</artifactId>
-				<version>3.0.1</version>
-				<inherited>true</inherited>
-				<extensions>true</extensions>
-			</plugin>
-		</plugins>
-	</build>
-	
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index aef7116..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
+++ /dev/null
@@ -1,552 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.common.state.ListState;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
-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.KafkaTopicPartitionState;
-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.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-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,
-		ResultTypeQueryable<T>,
-		CheckpointedFunction {
-	private static final long serialVersionUID = -6272159445203409112L;
-
-	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;
-
-	/** Boolean configuration key to disable metrics tracking **/
-	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
-
-	// ------------------------------------------------------------------------
-	//  configuration state, set on the client relevant for all subtasks
-	// ------------------------------------------------------------------------
-
-	private final List<String> topics;
-	
-	/** The schema to convert between Kafka's byte messages, and Flink's objects */
-	protected final KeyedDeserializationSchema<T> deserializer;
-
-	/** The set of topic partitions that the source will read */
-	protected List<KafkaTopicPartition> subscribedPartitions;
-	
-	/** 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;
-
-	private transient ListState<Tuple2<KafkaTopicPartition, Long>> offsetsStateForCheckpoint;
-
-	// ------------------------------------------------------------------------
-	//  runtime state (used individually by each parallel subtask) 
-	// ------------------------------------------------------------------------
-	
-	/** Data for pending but uncommitted offsets */
-	private final LinkedMap pendingOffsetsToCommit = 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 */
-	private transient volatile HashMap<KafkaTopicPartition, Long> restoreToOffset;
-	
-	/** Flag indicating whether the consumer is still running **/
-	private volatile boolean running = true;
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Base constructor.
-	 *
-	 * @param deserializer
-	 *           The deserializer to turn raw byte messages into Java/Scala objects.
-	 */
-	public FlinkKafkaConsumerBase(List<String> topics, KeyedDeserializationSchema<T> deserializer) {
-		this.topics = checkNotNull(topics);
-		checkArgument(topics.size() > 0, "You have to define at least one topic.");
-		this.deserializer = checkNotNull(deserializer, "valueDeserializer");
-	}
-
-	/**
-	 * 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.
-	 */
-	protected void setSubscribedPartitions(List<KafkaTopicPartition> allSubscribedPartitions) {
-		checkNotNull(allSubscribedPartitions);
-		this.subscribedPartitions = Collections.unmodifiableList(allSubscribedPartitions);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Configuration
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * 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> assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks<T> assigner) {
-		checkNotNull(assigner);
-		
-		if (this.periodicWatermarkAssigner != null) {
-			throw new IllegalStateException("A periodic watermark emitter has already been set.");
-		}
-		try {
-			ClosureCleaner.clean(assigner, true);
-			this.punctuatedWatermarkAssigner = new SerializedValue<>(assigner);
-			return this;
-		} catch (Exception e) {
-			throw new IllegalArgumentException("The given assigner is not serializable", e);
-		}
-	}
-
-	/**
-	 * 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> assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks<T> assigner) {
-		checkNotNull(assigner);
-		
-		if (this.punctuatedWatermarkAssigner != null) {
-			throw new IllegalStateException("A punctuated watermark emitter has already been set.");
-		}
-		try {
-			ClosureCleaner.clean(assigner, true);
-			this.periodicWatermarkAssigner = new SerializedValue<>(assigner);
-			return this;
-		} catch (Exception e) {
-			throw new IllegalArgumentException("The given assigner is not serializable", e);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Work methods
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void run(SourceContext<T> sourceContext) throws Exception {
-		if (subscribedPartitions == null) {
-			throw new Exception("The partitions were not set for the consumer");
-		}
-
-		// we need only do work, if we actually have partitions assigned
-		if (!subscribedPartitions.isEmpty()) {
-
-			// (1) create the fetcher that will communicate with the Kafka brokers
-			final AbstractFetcher<T, ?> fetcher = createFetcher(
-					sourceContext, subscribedPartitions,
-					periodicWatermarkAssigner, punctuatedWatermarkAssigner,
-					(StreamingRuntimeContext) getRuntimeContext());
-
-			// (2) set the fetcher to the restored checkpoint offsets
-			if (restoreToOffset != null) {
-				fetcher.restoreOffsets(restoreToOffset);
-			}
-
-			// 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();
-		}
-		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();
-					}
-				}
-			}
-		}
-	}
-
-	@Override
-	public void cancel() {
-		// set ourselves as not running
-		running = false;
-		
-		// abort the fetcher, if there is one
-		if (kafkaFetcher != null) {
-			kafkaFetcher.cancel();
-		}
-
-		// there will be an interrupt() call to the main thread anyways
-	}
-
-	@Override
-	public void open(Configuration configuration) {
-		List<KafkaTopicPartition> kafkaTopicPartitions = getKafkaPartitions(topics);
-
-		if (kafkaTopicPartitions != null) {
-			assignTopicPartitions(kafkaTopicPartitions);
-		}
-	}
-
-	@Override
-	public void close() throws Exception {
-		// pretty much the same logic as cancelling
-		try {
-			cancel();
-		} finally {
-			super.close();
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Checkpoint and restore
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void initializeState(FunctionInitializationContext context) throws Exception {
-
-		OperatorStateStore stateStore = context.getOperatorStateStore();
-		offsetsStateForCheckpoint = stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME);
-
-		if (context.isRestored()) {
-			restoreToOffset = new HashMap<>();
-			for (Tuple2<KafkaTopicPartition, Long> kafkaOffset : offsetsStateForCheckpoint.get()) {
-				restoreToOffset.put(kafkaOffset.f0, kafkaOffset.f1);
-			}
-
-			LOG.info("Setting restore state in the FlinkKafkaConsumer.");
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Using the following offsets: {}", restoreToOffset);
-			}
-		} else {
-			LOG.info("No restore state for FlinkKafkaConsumer.");
-		}
-	}
-
-	@Override
-	public void snapshotState(FunctionSnapshotContext context) throws Exception {
-		if (!running) {
-			LOG.debug("snapshotState() called on closed source");
-		} else {
-
-			offsetsStateForCheckpoint.clear();
-
-			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 or the assigned partitions
-
-				if (restoreToOffset != null) {
-
-					for (Map.Entry<KafkaTopicPartition, Long> kafkaTopicPartitionLongEntry : restoreToOffset.entrySet()) {
-						offsetsStateForCheckpoint.add(
-								Tuple2.of(kafkaTopicPartitionLongEntry.getKey(), kafkaTopicPartitionLongEntry.getValue()));
-					}
-				} else if (subscribedPartitions != null) {
-					for (KafkaTopicPartition subscribedPartition : subscribedPartitions) {
-						offsetsStateForCheckpoint.add(
-								Tuple2.of(subscribedPartition, KafkaTopicPartitionState.OFFSET_NOT_SET));
-					}
-				}
-
-				// the map cannot be asynchronously updated, because only one checkpoint call can happen
-				// on this function at a time: either snapshotState() or notifyCheckpointComplete()
-				pendingOffsetsToCommit.put(context.getCheckpointId(), restoreToOffset);
-			} else {
-				HashMap<KafkaTopicPartition, Long> currentOffsets = fetcher.snapshotCurrentState();
-
-				// the map cannot be asynchronously updated, because only one checkpoint call can happen
-				// on this function at a time: either snapshotState() or notifyCheckpointComplete()
-				pendingOffsetsToCommit.put(context.getCheckpointId(), currentOffsets);
-
-				for (Map.Entry<KafkaTopicPartition, Long> kafkaTopicPartitionLongEntry : currentOffsets.entrySet()) {
-					offsetsStateForCheckpoint.add(
-							Tuple2.of(kafkaTopicPartitionLongEntry.getKey(), kafkaTopicPartitionLongEntry.getValue()));
-				}
-			}
-
-			// truncate the map of pending offsets to commit, to prevent infinite growth
-			while (pendingOffsetsToCommit.size() > MAX_NUM_PENDING_CHECKPOINTS) {
-				pendingOffsetsToCommit.remove(0);
-			}
-		}
-	}
-
-	@Override
-	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		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 to Kafka/ZooKeeper for checkpoint " + checkpointId);
-		}
-
-		try {
-			final int posInMap = pendingOffsetsToCommit.indexOf(checkpointId);
-			if (posInMap == -1) {
-				LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
-				return;
-			}
-
-			@SuppressWarnings("unchecked")
-			HashMap<KafkaTopicPartition, Long> offsets =
-					(HashMap<KafkaTopicPartition, Long>) pendingOffsetsToCommit.remove(posInMap);
-
-			// remove older checkpoints in map
-			for (int i = 0; i < posInMap; i++) {
-				pendingOffsetsToCommit.remove(0);
-			}
-
-			if (offsets == null || offsets.size() == 0) {
-				LOG.debug("Checkpoint state was empty.");
-				return;
-			}
-			fetcher.commitInternalOffsetsToKafka(offsets);
-		}
-		catch (Exception e) {
-			if (running) {
-				throw e;
-			}
-			// else ignore exception if we are no longer running
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  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;
-
-	protected abstract List<KafkaTopicPartition> getKafkaPartitions(List<String> topics);
-	
-	// ------------------------------------------------------------------------
-	//  ResultTypeQueryable methods 
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public TypeInformation<T> getProducedType() {
-		return deserializer.getProducedType();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private void assignTopicPartitions(List<KafkaTopicPartition> kafkaTopicPartitions) {
-		subscribedPartitions = new ArrayList<>();
-
-		if (restoreToOffset != null) {
-			for (KafkaTopicPartition kafkaTopicPartition : kafkaTopicPartitions) {
-				if (restoreToOffset.containsKey(kafkaTopicPartition)) {
-					subscribedPartitions.add(kafkaTopicPartition);
-				}
-			}
-		} else {
-			Collections.sort(kafkaTopicPartitions, new Comparator<KafkaTopicPartition>() {
-				@Override
-				public int compare(KafkaTopicPartition o1, KafkaTopicPartition o2) {
-					int topicComparison = o1.getTopic().compareTo(o2.getTopic());
-
-					if (topicComparison == 0) {
-						return o1.getPartition() - o2.getPartition();
-					} else {
-						return topicComparison;
-					}
-				}
-			});
-
-			for (int i = getRuntimeContext().getIndexOfThisSubtask(); i < kafkaTopicPartitions.size(); i += getRuntimeContext().getNumberOfParallelSubtasks()) {
-				subscribedPartitions.add(kafkaTopicPartitions.get(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) {
-				thisSubtaskPartitions.add(allPartitions.get(i));
-			}
-		}
-		
-		return thisSubtaskPartitions;
-	}
-	
-	/**
-	 * Logs the partition information in INFO level.
-	 * 
-	 * @param logger The logger to log to.
-	 * @param partitionInfos List of subscribed partitions
-	 */
-	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());
-			if (count == null) {
-				count = 1;
-			} else {
-				count++;
-			}
-			countPerTopic.put(partition.getTopic(), count);
-		}
-		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("), ");
-		}
-		
-		logger.info(sb.toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
deleted file mode 100644
index d413f1c..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
+++ /dev/null
@@ -1,386 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.runtime.util.SerializableObject;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.util.NetUtils;
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Collections;
-import java.util.Comparator;
-
-import static java.util.Objects.requireNonNull;
-
-
-/**
- * Flink Sink to produce data into a Kafka topic.
- *
- * Please note that this producer provides at-least-once reliability guarantees when
- * checkpoints are enabled and setFlushOnCheckpoint(true) is set.
- * Otherwise, the producer doesn't provide any reliability guarantees.
- *
- * @param <IN> Type of the messages to write into Kafka.
- */
-public abstract class FlinkKafkaProducerBase<IN> extends RichSinkFunction<IN> implements CheckpointedFunction {
-
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
-
-	private static final long serialVersionUID = 1L;
-
-	/**
-	 * Configuration key for disabling the metrics reporting
-	 */
-	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
-
-	/**
-	 * Array with the partition ids of the given defaultTopicId
-	 * The size of this array is the number of partitions
-	 */
-	protected int[] partitions;
-
-	/**
-	 * User defined properties for the Producer
-	 */
-	protected final Properties producerConfig;
-
-	/**
-	 * The name of the default topic this producer is writing data to
-	 */
-	protected final String defaultTopicId;
-
-	/**
-	 * (Serializable) SerializationSchema for turning objects used with Flink into
-	 * byte[] for Kafka.
-	 */
-	protected final KeyedSerializationSchema<IN> schema;
-
-	/**
-	 * User-provided partitioner for assigning an object to a Kafka partition.
-	 */
-	protected final KafkaPartitioner<IN> partitioner;
-
-	/**
-	 * Flag indicating whether to accept failures (and log them), or to fail on failures
-	 */
-	protected boolean logFailuresOnly;
-
-	/**
-	 * If true, the producer will wait until all outstanding records have been send to the broker.
-	 */
-	protected boolean flushOnCheckpoint;
-	
-	// -------------------------------- Runtime fields ------------------------------------------
-
-	/** KafkaProducer instance */
-	protected transient KafkaProducer<byte[], byte[]> producer;
-
-	/** The callback than handles error propagation or logging callbacks */
-	protected transient Callback callback;
-
-	/** Errors encountered in the async producer are stored here */
-	protected transient volatile Exception asyncException;
-
-	/** Lock for accessing the pending records */
-	protected final SerializableObject pendingRecordsLock = new SerializableObject();
-
-	/** Number of unacknowledged records. */
-	protected long pendingRecords;
-
-	protected OperatorStateStore stateStore;
-
-
-	/**
-	 * The main constructor for creating a FlinkKafkaProducer.
-	 *
-	 * @param defaultTopicId The default topic to write data to
-	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
-	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
-	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner
-	 */
-	public FlinkKafkaProducerBase(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, KafkaPartitioner<IN> customPartitioner) {
-		requireNonNull(defaultTopicId, "TopicID not set");
-		requireNonNull(serializationSchema, "serializationSchema not set");
-		requireNonNull(producerConfig, "producerConfig not set");
-		ClosureCleaner.clean(customPartitioner, true);
-		ClosureCleaner.ensureSerializable(serializationSchema);
-
-		this.defaultTopicId = defaultTopicId;
-		this.schema = serializationSchema;
-		this.producerConfig = producerConfig;
-
-		// set the producer configuration properties for kafka record key value serializers.
-		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
-			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
-		} else {
-			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
-		}
-
-		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
-			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
-		} else {
-			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
-		}
-
-		// eagerly ensure that bootstrap servers are set.
-		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
-			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
-		}
-
-		this.partitioner = customPartitioner;
-	}
-
-	// ---------------------------------- Properties --------------------------
-
-	/**
-	 * Defines whether the producer should fail on errors, or only log them.
-	 * If this is set to true, then exceptions will be only logged, if set to false,
-	 * exceptions will be eventually thrown and cause the streaming program to 
-	 * fail (and enter recovery).
-	 * 
-	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
-	 */
-	public void setLogFailuresOnly(boolean logFailuresOnly) {
-		this.logFailuresOnly = logFailuresOnly;
-	}
-
-	/**
-	 * If set to true, the Flink producer will wait for all outstanding messages in the Kafka buffers
-	 * to be acknowledged by the Kafka producer on a checkpoint.
-	 * This way, the producer can guarantee that messages in the Kafka buffers are part of the checkpoint.
-	 *
-	 * @param flush Flag indicating the flushing mode (true = flush on checkpoint)
-	 */
-	public void setFlushOnCheckpoint(boolean flush) {
-		this.flushOnCheckpoint = flush;
-	}
-
-	/**
-	 * Used for testing only
-	 */
-	protected <K,V> KafkaProducer<K,V> getKafkaProducer(Properties props) {
-		return new KafkaProducer<>(props);
-	}
-
-	// ----------------------------------- Utilities --------------------------
-	
-	/**
-	 * Initializes the connection to Kafka.
-	 */
-	@Override
-	public void open(Configuration configuration) {
-		producer = getKafkaProducer(this.producerConfig);
-
-		// the fetched list is immutable, so we're creating a mutable copy in order to sort it
-		List<PartitionInfo> partitionsList = new ArrayList<>(producer.partitionsFor(defaultTopicId));
-
-		// sort the partitions by partition id to make sure the fetched partition list is the same across subtasks
-		Collections.sort(partitionsList, new Comparator<PartitionInfo>() {
-			@Override
-			public int compare(PartitionInfo o1, PartitionInfo o2) {
-				return Integer.compare(o1.partition(), o2.partition());
-			}
-		});
-
-		partitions = new int[partitionsList.size()];
-		for (int i = 0; i < partitions.length; i++) {
-			partitions[i] = partitionsList.get(i).partition();
-		}
-
-		RuntimeContext ctx = getRuntimeContext();
-		if (partitioner != null) {
-			partitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), partitions);
-		}
-
-		LOG.info("Starting FlinkKafkaProducer ({}/{}) to produce into topic {}", 
-				ctx.getIndexOfThisSubtask() + 1, ctx.getNumberOfParallelSubtasks(), defaultTopicId);
-
-		// register Kafka metrics to Flink accumulators
-		if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) {
-			Map<MetricName, ? extends Metric> metrics = this.producer.metrics();
-
-			if (metrics == null) {
-				// MapR's Kafka implementation returns null here.
-				LOG.info("Producer implementation does not support metrics");
-			} else {
-				final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer");
-				for (Map.Entry<MetricName, ? extends Metric> metric: metrics.entrySet()) {
-					kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue()));
-				}
-			}
-		}
-
-		if (flushOnCheckpoint && !((StreamingRuntimeContext)this.getRuntimeContext()).isCheckpointingEnabled()) {
-			LOG.warn("Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing.");
-			flushOnCheckpoint = false;
-		}
-
-		if (logFailuresOnly) {
-			callback = new Callback() {
-				@Override
-				public void onCompletion(RecordMetadata metadata, Exception e) {
-					if (e != null) {
-						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
-					}
-					acknowledgeMessage();
-				}
-			};
-		}
-		else {
-			callback = new Callback() {
-				@Override
-				public void onCompletion(RecordMetadata metadata, Exception exception) {
-					if (exception != null && asyncException == null) {
-						asyncException = exception;
-					}
-					acknowledgeMessage();
-				}
-			};
-		}
-	}
-
-	/**
-	 * Called when new data arrives to the sink, and forwards it to Kafka.
-	 *
-	 * @param next
-	 * 		The incoming data
-	 */
-	@Override
-	public void invoke(IN next) throws Exception {
-		// propagate asynchronous errors
-		checkErroneous();
-
-		byte[] serializedKey = schema.serializeKey(next);
-		byte[] serializedValue = schema.serializeValue(next);
-		String targetTopic = schema.getTargetTopic(next);
-		if (targetTopic == null) {
-			targetTopic = defaultTopicId;
-		}
-
-		ProducerRecord<byte[], byte[]> record;
-		if (partitioner == null) {
-			record = new ProducerRecord<>(targetTopic, serializedKey, serializedValue);
-		} else {
-			record = new ProducerRecord<>(targetTopic, partitioner.partition(next, serializedKey, serializedValue, partitions.length), serializedKey, serializedValue);
-		}
-		if (flushOnCheckpoint) {
-			synchronized (pendingRecordsLock) {
-				pendingRecords++;
-			}
-		}
-		producer.send(record, callback);
-	}
-
-
-	@Override
-	public void close() throws Exception {
-		if (producer != null) {
-			producer.close();
-		}
-		
-		// make sure we propagate pending errors
-		checkErroneous();
-	}
-
-	// ------------------- Logic for handling checkpoint flushing -------------------------- //
-
-	private void acknowledgeMessage() {
-		if (flushOnCheckpoint) {
-			synchronized (pendingRecordsLock) {
-				pendingRecords--;
-				if (pendingRecords == 0) {
-					pendingRecordsLock.notifyAll();
-				}
-			}
-		}
-	}
-
-	/**
-	 * Flush pending records.
-	 */
-	protected abstract void flush();
-
-	@Override
-	public void initializeState(FunctionInitializationContext context) throws Exception {
-		this.stateStore = context.getOperatorStateStore();
-	}
-
-	@Override
-	public void snapshotState(FunctionSnapshotContext ctx) throws Exception {
-		if (flushOnCheckpoint) {
-			// flushing is activated: We need to wait until pendingRecords is 0
-			flush();
-			synchronized (pendingRecordsLock) {
-				if (pendingRecords != 0) {
-					throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecords);
-				}
-				// pending records count is 0. We can now confirm the checkpoint
-			}
-		}
-	}
-
-	// ----------------------------------- Utilities --------------------------
-
-	protected void checkErroneous() throws Exception {
-		Exception e = asyncException;
-		if (e != null) {
-			// prevent double throwing
-			asyncException = null;
-			throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e);
-		}
-	}
-	
-	public static Properties getPropertiesFromBrokerList(String brokerList) {
-		String[] elements = brokerList.split(",");
-		
-		// validate the broker addresses
-		for (String broker: elements) {
-			NetUtils.getCorrectHostnamePort(broker);
-		}
-		
-		Properties props = new Properties();
-		props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList);
-		return props;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
deleted file mode 100644
index ee98783..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-import java.util.Properties;
-
-/**
- * Base class for {@link KafkaTableSink} that serializes data in JSON format
- */
-public abstract class KafkaJsonTableSink extends KafkaTableSink {
-
-	/**
-	 * Creates KafkaJsonTableSink
-	 *
-	 * @param topic topic in Kafka to which table is written
-	 * @param properties properties to connect to Kafka
-	 * @param partitioner Kafka partitioner
-	 */
-	public KafkaJsonTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner) {
-		super(topic, properties, partitioner);
-	}
-
-	@Override
-	protected SerializationSchema<Row> createSerializationSchema(String[] fieldNames) {
-		return new JsonRowSerializationSchema(fieldNames);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
deleted file mode 100644
index f145509..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.sources.StreamTableSource;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * A version-agnostic Kafka JSON {@link StreamTableSource}.
- *
- * <p>The version-specific Kafka consumers need to extend this class and
- * override {@link #getKafkaConsumer(String, Properties, DeserializationSchema)}}.
- *
- * <p>The field names are used to parse the JSON file and so are the types.
- */
-public abstract class KafkaJsonTableSource extends KafkaTableSource {
-
-	/**
-	 * Creates a generic Kafka JSON {@link StreamTableSource}.
-	 *
-	 * @param topic      Kafka topic to consume.
-	 * @param properties Properties for the Kafka consumer.
-	 * @param fieldNames Row field names.
-	 * @param fieldTypes Row field types.
-	 */
-	KafkaJsonTableSource(
-			String topic,
-			Properties properties,
-			String[] fieldNames,
-			Class<?>[] fieldTypes) {
-
-		super(topic, properties, createDeserializationSchema(fieldNames, fieldTypes), fieldNames, fieldTypes);
-	}
-
-	/**
-	 * Creates a generic Kafka JSON {@link StreamTableSource}.
-	 *
-	 * @param topic      Kafka topic to consume.
-	 * @param properties Properties for the Kafka consumer.
-	 * @param fieldNames Row field names.
-	 * @param fieldTypes Row field types.
-	 */
-	KafkaJsonTableSource(
-			String topic,
-			Properties properties,
-			String[] fieldNames,
-			TypeInformation<?>[] fieldTypes) {
-
-		super(topic, properties, createDeserializationSchema(fieldNames, fieldTypes), fieldNames, fieldTypes);
-	}
-
-	/**
-	 * Configures the failure behaviour if a JSON field is missing.
-	 *
-	 * <p>By default, a missing field is ignored and the field is set to null.
-	 *
-	 * @param failOnMissingField Flag indicating whether to fail or not on a missing field.
-	 */
-	public void setFailOnMissingField(boolean failOnMissingField) {
-		JsonRowDeserializationSchema deserializationSchema = (JsonRowDeserializationSchema) getDeserializationSchema();
-		deserializationSchema.setFailOnMissingField(failOnMissingField);
-	}
-
-	private static JsonRowDeserializationSchema createDeserializationSchema(
-			String[] fieldNames,
-			TypeInformation<?>[] fieldTypes) {
-
-		return new JsonRowDeserializationSchema(fieldNames, fieldTypes);
-	}
-
-	private static JsonRowDeserializationSchema createDeserializationSchema(
-			String[] fieldNames,
-			Class<?>[] fieldTypes) {
-
-		return new JsonRowDeserializationSchema(fieldNames, fieldTypes);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
deleted file mode 100644
index 714d9cd..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.sinks.StreamTableSink;
-import org.apache.flink.api.table.typeutils.RowTypeInfo;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Properties;
-
-/**
- * A version-agnostic Kafka {@link StreamTableSink}.
- *
- * <p>The version-specific Kafka consumers need to extend this class and
- * override {@link #createKafkaProducer(String, Properties, SerializationSchema, KafkaPartitioner)}}.
- */
-public abstract class KafkaTableSink implements StreamTableSink<Row> {
-
-	protected final String topic;
-	protected final Properties properties;
-	protected SerializationSchema<Row> serializationSchema;
-	protected final KafkaPartitioner<Row> partitioner;
-	protected String[] fieldNames;
-	protected TypeInformation[] fieldTypes;
-
-	/**
-	 * Creates KafkaTableSink
-	 *
-	 * @param topic                 Kafka topic to write to.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param partitioner           Partitioner to select Kafka partition for each item
-	 */
-	public KafkaTableSink(
-			String topic,
-			Properties properties,
-			KafkaPartitioner<Row> partitioner) {
-
-		this.topic = Preconditions.checkNotNull(topic, "topic");
-		this.properties = Preconditions.checkNotNull(properties, "properties");
-		this.partitioner = Preconditions.checkNotNull(partitioner, "partitioner");
-	}
-
-	/**
-	 * Returns the version-specifid Kafka producer.
-	 *
-	 * @param topic               Kafka topic to produce to.
-	 * @param properties          Properties for the Kafka producer.
-	 * @param serializationSchema Serialization schema to use to create Kafka records.
-	 * @param partitioner         Partitioner to select Kafka partition.
-	 * @return The version-specific Kafka producer
-	 */
-	protected abstract FlinkKafkaProducerBase<Row> createKafkaProducer(
-		String topic, Properties properties,
-		SerializationSchema<Row> serializationSchema,
-		KafkaPartitioner<Row> partitioner);
-
-	/**
-	 * Create serialization schema for converting table rows into bytes.
-	 *
-	 * @param fieldNames Field names in table rows.
-	 * @return Instance of serialization schema
-	 */
-	protected abstract SerializationSchema<Row> createSerializationSchema(String[] fieldNames);
-
-	/**
-	 * Create a deep copy of this sink.
-	 *
-	 * @return Deep copy of this sink
-	 */
-	protected abstract KafkaTableSink createCopy();
-
-	@Override
-	public void emitDataStream(DataStream<Row> dataStream) {
-		FlinkKafkaProducerBase<Row> kafkaProducer = createKafkaProducer(topic, properties, serializationSchema, partitioner);
-		dataStream.addSink(kafkaProducer);
-	}
-
-	@Override
-	public TypeInformation<Row> getOutputType() {
-		return new RowTypeInfo(getFieldTypes());
-	}
-
-	public String[] getFieldNames() {
-		return fieldNames;
-	}
-
-	@Override
-	public TypeInformation<?>[] getFieldTypes() {
-		return fieldTypes;
-	}
-
-	@Override
-	public KafkaTableSink configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
-		KafkaTableSink copy = createCopy();
-		copy.fieldNames = Preconditions.checkNotNull(fieldNames, "fieldNames");
-		copy.fieldTypes = Preconditions.checkNotNull(fieldTypes, "fieldTypes");
-		Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
-			"Number of provided field names and types does not match.");
-		copy.serializationSchema = createSerializationSchema(fieldNames);
-
-		return copy;
-	}
-
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
deleted file mode 100644
index fd423d7..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
-import org.apache.flink.api.table.typeutils.RowTypeInfo;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Properties;
-
-import static org.apache.flink.streaming.connectors.kafka.internals.TypeUtil.toTypeInfo;
-
-/**
- * A version-agnostic Kafka {@link StreamTableSource}.
- *
- * <p>The version-specific Kafka consumers need to extend this class and
- * override {@link #getKafkaConsumer(String, Properties, DeserializationSchema)}}.
- */
-public abstract class KafkaTableSource implements StreamTableSource<Row> {
-
-	/** The Kafka topic to consume. */
-	private final String topic;
-
-	/** Properties for the Kafka consumer. */
-	private final Properties properties;
-
-	/** Deserialization schema to use for Kafka records. */
-	private final DeserializationSchema<Row> deserializationSchema;
-
-	/** Row field names. */
-	private final String[] fieldNames;
-
-	/** Row field types. */
-	private final TypeInformation<?>[] fieldTypes;
-
-	/**
-	 * Creates a generic Kafka {@link StreamTableSource}.
-	 *
-	 * @param topic                 Kafka topic to consume.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param deserializationSchema Deserialization schema to use for Kafka records.
-	 * @param fieldNames            Row field names.
-	 * @param fieldTypes            Row field types.
-	 */
-	KafkaTableSource(
-			String topic,
-			Properties properties,
-			DeserializationSchema<Row> deserializationSchema,
-			String[] fieldNames,
-			Class<?>[] fieldTypes) {
-
-		this(topic, properties, deserializationSchema, fieldNames, toTypeInfo(fieldTypes));
-	}
-
-	/**
-	 * Creates a generic Kafka {@link StreamTableSource}.
-	 *
-	 * @param topic                 Kafka topic to consume.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param deserializationSchema Deserialization schema to use for Kafka records.
-	 * @param fieldNames            Row field names.
-	 * @param fieldTypes            Row field types.
-	 */
-	KafkaTableSource(
-			String topic,
-			Properties properties,
-			DeserializationSchema<Row> deserializationSchema,
-			String[] fieldNames,
-			TypeInformation<?>[] fieldTypes) {
-
-		this.topic = Preconditions.checkNotNull(topic, "Topic");
-		this.properties = Preconditions.checkNotNull(properties, "Properties");
-		this.deserializationSchema = Preconditions.checkNotNull(deserializationSchema, "Deserialization schema");
-		this.fieldNames = Preconditions.checkNotNull(fieldNames, "Field names");
-		this.fieldTypes = Preconditions.checkNotNull(fieldTypes, "Field types");
-
-		Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
-				"Number of provided field names and types does not match.");
-	}
-
-	/**
-	 * NOTE: This method is for internal use only for defining a TableSource.
-	 *       Do not use it in Table API programs.
-	 */
-	@Override
-	public DataStream<Row> getDataStream(StreamExecutionEnvironment env) {
-		// Version-specific Kafka consumer
-		FlinkKafkaConsumerBase<Row> kafkaConsumer = getKafkaConsumer(topic, properties, deserializationSchema);
-		DataStream<Row> kafkaSource = env.addSource(kafkaConsumer);
-		return kafkaSource;
-	}
-
-	@Override
-	public int getNumberOfFields() {
-		return fieldNames.length;
-	}
-
-	@Override
-	public String[] getFieldsNames() {
-		return fieldNames;
-	}
-
-	@Override
-	public TypeInformation<?>[] getFieldTypes() {
-		return fieldTypes;
-	}
-
-	@Override
-	public TypeInformation<Row> getReturnType() {
-		return new RowTypeInfo(fieldTypes);
-	}
-
-	/**
-	 * Returns the version-specific Kafka consumer.
-	 *
-	 * @param topic                 Kafka topic to consume.
-	 * @param properties            Properties for the Kafka consumer.
-	 * @param deserializationSchema Deserialization schema to use for Kafka records.
-	 * @return The version-specific Kafka consumer
-	 */
-	abstract FlinkKafkaConsumerBase<Row> getKafkaConsumer(
-			String topic,
-			Properties properties,
-			DeserializationSchema<Row> deserializationSchema);
-
-	/**
-	 * Returns the deserialization schema.
-	 *
-	 * @return The deserialization schema
-	 */
-	protected DeserializationSchema<Row> getDeserializationSchema() {
-		return deserializationSchema;
-	}
-}


[09/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index d495327..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.metrics.MetricGroup;
-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.connectors.kafka.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-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.OffsetAndMetadata;
-import org.apache.kafka.common.TopicPartition;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-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> {
-
-	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 handover of data and exceptions between the consumer thread and the task thread */
-	private final Handover handover;
-
-	/** The thread that runs the actual KafkaConsumer and hand the record batches to this fetcher */
-	private final KafkaConsumerThread consumerThread;
-
-	/** 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,
-			ProcessingTimeService processingTimeProvider,
-			long autoWatermarkInterval,
-			ClassLoader userCodeClassLoader,
-			boolean enableCheckpointing,
-			String taskNameWithSubtasks,
-			MetricGroup metricGroup,
-			KeyedDeserializationSchema<T> deserializer,
-			Properties kafkaProperties,
-			long pollTimeout,
-			boolean useMetrics) throws Exception
-	{
-		super(
-				sourceContext,
-				assignedPartitions,
-				watermarksPeriodic,
-				watermarksPunctuated,
-				processingTimeProvider,
-				autoWatermarkInterval,
-				userCodeClassLoader,
-				useMetrics);
-
-		this.deserializer = deserializer;
-		this.handover = new Handover();
-
-		final MetricGroup kafkaMetricGroup = metricGroup.addGroup("KafkaConsumer");
-		addOffsetStateGauge(kafkaMetricGroup);
-
-		// if checkpointing is enabled, we are not automatically committing to Kafka.
-		kafkaProperties.setProperty(
-				ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG,
-				Boolean.toString(!enableCheckpointing));
-		
-		this.consumerThread = new KafkaConsumerThread(
-				LOG,
-				handover,
-				kafkaProperties,
-				subscribedPartitions(),
-				kafkaMetricGroup,
-				createCallBridge(),
-				getFetcherName() + " for " + taskNameWithSubtasks,
-				pollTimeout,
-				useMetrics);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Fetcher work methods
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void runFetchLoop() throws Exception {
-		try {
-			final Handover handover = this.handover;
-
-			// kick off the actual Kafka consumer
-			consumerThread.start();
-
-			while (running) {
-				// this blocks until we get the next records
-				// it automatically re-throws exceptions encountered in the fetcher thread
-				final ConsumerRecords<byte[], byte[]> records = handover.pollNext();
-
-				// 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) {
-
-						final 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 updates offset state atomically
-						// and deals with timestamps and watermark generation
-						emitRecord(value, partition, record.offset(), record);
-					}
-				}
-			}
-		}
-		finally {
-			// this signals the consumer thread that no more work is to be done
-			consumerThread.shutdown();
-		}
-
-		// on a clean exit, wait for the runner thread
-		try {
-			consumerThread.join();
-		}
-		catch (InterruptedException e) {
-			// may be the result of a wake-up interruption after an exception.
-			// we ignore this here and only restore the interruption state
-			Thread.currentThread().interrupt();
-		}
-	}
-
-	@Override
-	public void cancel() {
-		// flag the main thread to exit. A thread interrupt will come anyways.
-		running = false;
-		handover.close();
-		consumerThread.shutdown();
-	}
-
-	// ------------------------------------------------------------------------
-	//  The below methods are overridden in the 0.10 fetcher, which otherwise
-	//   reuses most of the 0.9 fetcher behavior
-	// ------------------------------------------------------------------------
-
-	protected void emitRecord(
-			T record,
-			KafkaTopicPartitionState<TopicPartition> partition,
-			long offset,
-			@SuppressWarnings("UnusedParameters") ConsumerRecord<?, ?> consumerRecord) throws Exception {
-
-		// the 0.9 Fetcher does not try to extract a timestamp
-		emitRecord(record, partition, offset);
-	}
-
-	/**
-	 * Gets the name of this fetcher, for thread naming and logging purposes.
-	 */
-	protected String getFetcherName() {
-		return "Kafka 0.9 Fetcher";
-	}
-
-	protected KafkaConsumerCallBridge createCallBridge() {
-		return new KafkaConsumerCallBridge();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Implement Methods of the AbstractFetcher
-	// ------------------------------------------------------------------------
-
-	@Override
-	public TopicPartition createKafkaPartitionHandle(KafkaTopicPartition partition) {
-		return new TopicPartition(partition.getTopic(), partition.getPartition());
-	}
-
-	@Override
-	public void commitInternalOffsetsToKafka(Map<KafkaTopicPartition, Long> offsets) throws Exception {
-		KafkaTopicPartitionState<TopicPartition>[] partitions = subscribedPartitions();
-		Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = new HashMap<>(partitions.length);
-
-		for (KafkaTopicPartitionState<TopicPartition> partition : partitions) {
-			Long lastProcessedOffset = offsets.get(partition.getKafkaTopicPartition());
-			if (lastProcessedOffset != null) {
-				// committed offsets through the KafkaConsumer need to be 1 more than the last processed offset.
-				// This does not affect Flink's checkpoints/saved state.
-				long offsetToCommit = lastProcessedOffset + 1;
-
-				offsetsToCommit.put(partition.getKafkaPartitionHandle(), new OffsetAndMetadata(offsetToCommit));
-				partition.setCommittedOffset(offsetToCommit);
-			}
-		}
-
-		// record the work to be committed by the main consumer thread and make sure the consumer notices that
-		consumerThread.setOffsetsToCommit(offsetsToCommit);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java
deleted file mode 100644
index c17aae6..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internal;
-
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.List;
-
-/**
- * The ConsumerCallBridge simply calls methods on the {@link KafkaConsumer}.
- * 
- * This indirection is necessary, because Kafka broke binary compatibility between 0.9 and 0.10,
- * for example changing {@code assign(List)} to {@code assign(Collection)}.
- * 
- * Because of that, we need to two versions whose compiled code goes against different method signatures.
- * Even though the source of subclasses may look identical, the byte code will be different, because they
- * are compiled against different dependencies.
- */
-public class KafkaConsumerCallBridge {
-
-	public void assignPartitions(KafkaConsumer<?, ?> consumer, List<TopicPartition> topicPartitions) throws Exception {
-		consumer.assign(topicPartitions);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java
deleted file mode 100644
index 9cfa840..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java
+++ /dev/null
@@ -1,332 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.metrics.MetricGroup;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
-import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper;
-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.clients.consumer.OffsetCommitCallback;
-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 java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The thread the runs the {@link KafkaConsumer}, connecting to the brokers and polling records.
- * The thread pushes the data into a {@link Handover} to be picked up by the fetcher that will
- * deserialize and emit the records.
- * 
- * <p><b>IMPORTANT:</b> This thread must not be interrupted when attempting to shut it down.
- * The Kafka consumer code was found to not always handle interrupts well, and to even
- * deadlock in certain situations.
- * 
- * <p>Implementation Note: This code is written to be reusable in later versions of the KafkaConsumer.
- * Because Kafka is not maintaining binary compatibility, we use a "call bridge" as an indirection
- * to the KafkaConsumer calls that change signature.
- */
-public class KafkaConsumerThread extends Thread {
-
-	/** Logger for this consumer */
-	private final Logger log;
-
-	/** The handover of data and exceptions between the consumer thread and the task thread */
-	private final Handover handover;
-
-	/** The next offsets that the main thread should commit */
-	private final AtomicReference<Map<TopicPartition, OffsetAndMetadata>> nextOffsetsToCommit;
-
-	/** The configuration for the Kafka consumer */
-	private final Properties kafkaProperties;
-
-	/** The partitions that this consumer reads from */ 
-	private final KafkaTopicPartitionState<TopicPartition>[] subscribedPartitions;
-
-	/** We get this from the outside to publish metrics. **/
-	private final MetricGroup kafkaMetricGroup;
-
-	/** The indirections on KafkaConsumer methods, for cases where KafkaConsumer compatibility is broken */
-	private final KafkaConsumerCallBridge consumerCallBridge;
-
-	/** The maximum number of milliseconds to wait for a fetch batch */
-	private final long pollTimeout;
-
-	/** Flag whether to add Kafka's metrics to the Flink metrics */
-	private final boolean useMetrics;
-
-	/** Reference to the Kafka consumer, once it is created */
-	private volatile KafkaConsumer<byte[], byte[]> consumer;
-
-	/** Flag to mark the main work loop as alive */
-	private volatile boolean running;
-
-	/** Flag tracking whether the latest commit request has completed */
-	private volatile boolean commitInProgress;
-
-
-	public KafkaConsumerThread(
-			Logger log,
-			Handover handover,
-			Properties kafkaProperties,
-			KafkaTopicPartitionState<TopicPartition>[] subscribedPartitions,
-			MetricGroup kafkaMetricGroup,
-			KafkaConsumerCallBridge consumerCallBridge,
-			String threadName,
-			long pollTimeout,
-			boolean useMetrics) {
-
-		super(threadName);
-		setDaemon(true);
-
-		this.log = checkNotNull(log);
-		this.handover = checkNotNull(handover);
-		this.kafkaProperties = checkNotNull(kafkaProperties);
-		this.subscribedPartitions = checkNotNull(subscribedPartitions);
-		this.kafkaMetricGroup = checkNotNull(kafkaMetricGroup);
-		this.consumerCallBridge = checkNotNull(consumerCallBridge);
-		this.pollTimeout = pollTimeout;
-		this.useMetrics = useMetrics;
-
-		this.nextOffsetsToCommit = new AtomicReference<>();
-		this.running = true;
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void run() {
-		// early exit check
-		if (!running) {
-			return;
-		}
-
-		// this is the means to talk to FlinkKafkaConsumer's main thread
-		final Handover handover = this.handover;
-
-		// 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) {
-			handover.reportError(t);
-			return;
-		}
-
-		// from here on, the consumer is guaranteed to be closed properly
-		try {
-			// The callback invoked by Kafka once an offset commit is complete
-			final OffsetCommitCallback offsetCommitCallback = new CommitCallback();
-
-			// tell the consumer which partitions to work with
-			consumerCallBridge.assignPartitions(consumer, convertKafkaPartitions(subscribedPartitions));
-
-			// register Kafka's very own metrics in Flink's metric reporters
-			if (useMetrics) {
-				// register Kafka metrics to Flink
-				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 Kafka metrics, register them
-					for (Map.Entry<MetricName, ? extends Metric> metric: metrics.entrySet()) {
-						kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue()));
-					}
-				}
-			}
-
-			// early exit check
-			if (!running) {
-				return;
-			}
-
-			// seek the consumer to the initial offsets
-			for (KafkaTopicPartitionState<TopicPartition> partition : subscribedPartitions) {
-				if (partition.isOffsetDefined()) {
-					log.info("Partition {} has restored initial offsets {} from checkpoint / savepoint; " +
-							"seeking the consumer to position {}",
-							partition.getKafkaPartitionHandle(), partition.getOffset(), partition.getOffset() + 1);
-
-					consumer.seek(partition.getKafkaPartitionHandle(), partition.getOffset() + 1);
-				}
-				else {
-					// for partitions that do not have offsets restored from a checkpoint/savepoint,
-					// we need to define our internal offset state for them using the initial offsets retrieved from Kafka
-					// by the KafkaConsumer, so that they are correctly checkpointed and committed on the next checkpoint
-
-					long fetchedOffset = consumer.position(partition.getKafkaPartitionHandle());
-
-					log.info("Partition {} has no initial offset; the consumer has position {}, " +
-							"so the initial offset will be set to {}",
-							partition.getKafkaPartitionHandle(), fetchedOffset, fetchedOffset - 1);
-
-					// the fetched offset represents the next record to process, so we need to subtract it by 1
-					partition.setOffset(fetchedOffset - 1);
-				}
-			}
-
-			// from now on, external operations may call the consumer
-			this.consumer = consumer;
-
-			// the latest bulk of records. may carry across the loop if the thread is woken up
-			// from blocking on the handover
-			ConsumerRecords<byte[], byte[]> records = null;
-
-			// main fetch loop
-			while (running) {
-
-				// check if there is something to commit
-				if (!commitInProgress) {
-					// get and reset the work-to-be committed, so we don't repeatedly commit the same
-					final Map<TopicPartition, OffsetAndMetadata> toCommit = nextOffsetsToCommit.getAndSet(null);
-
-					if (toCommit != null) {
-						log.debug("Sending async offset commit request to Kafka broker");
-
-						// also record that a commit is already in progress
-						// the order here matters! first set the flag, then send the commit command.
-						commitInProgress = true;
-						consumer.commitAsync(toCommit, offsetCommitCallback);
-					}
-				}
-
-				// get the next batch of records, unless we did not manage to hand the old batch over
-				if (records == null) {
-					try {
-						records = consumer.poll(pollTimeout);
-					}
-					catch (WakeupException we) {
-						continue;
-					}
-				}
-
-				try {
-					handover.produce(records);
-					records = null;
-				}
-				catch (Handover.WakeupException e) {
-					// fall through the loop
-				}
-			}
-			// end main fetch loop
-		}
-		catch (Throwable t) {
-			// let the main thread know and exit
-			// it may be that this exception comes because the main thread closed the handover, in
-			// which case the below reporting is irrelevant, but does not hurt either
-			handover.reportError(t);
-		}
-		finally {
-			// make sure the handover is closed if it is not already closed or has an error
-			handover.close();
-
-			// make sure the KafkaConsumer is closed
-			try {
-				consumer.close();
-			}
-			catch (Throwable t) {
-				log.warn("Error while closing Kafka consumer", t);
-			}
-		}
-	}
-
-	/**
-	 * Shuts this thread down, waking up the thread gracefully if blocked (without Thread.interrupt() calls).
-	 */
-	public void shutdown() {
-		running = false;
-
-		// We cannot call close() on the KafkaConsumer, because it will actually throw
-		// an exception if a concurrent call is in progress
-
-		// this wakes up the consumer if it is blocked handing over records
-		handover.wakeupProducer();
-
-		// this wakes up the consumer if it is blocked in a kafka poll 
-		if (consumer != null) {
-			consumer.wakeup();
-		}
-	}
-
-	/**
-	 * Tells this thread to commit a set of offsets. This method does not block, the committing
-	 * operation will happen asynchronously.
-	 * 
-	 * <p>Only one commit operation may be pending at any time. If the committing takes longer than
-	 * the frequency with which this method is called, then some commits may be skipped due to being
-	 * superseded  by newer ones.
-	 * 
-	 * @param offsetsToCommit The offsets to commit
-	 */
-	public void setOffsetsToCommit(Map<TopicPartition, OffsetAndMetadata> offsetsToCommit) {
-		// record the work to be committed by the main consumer thread and make sure the consumer notices that
-		if (nextOffsetsToCommit.getAndSet(offsetsToCommit) != null) {
-			log.warn("Committing offsets to Kafka takes longer than the checkpoint interval. " +
-					"Skipping commit of previous offsets because newer complete checkpoint offsets are available. " +
-					"This does not compromise Flink's checkpoint integrity.");
-		}
-
-		// if the consumer is blocked in a poll() or handover operation, wake it up to commit soon
-		handover.wakeupProducer();
-		if (consumer != null) {
-			consumer.wakeup();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private 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;
-	}
-
-	// ------------------------------------------------------------------------
-
-	private class CommitCallback implements OffsetCommitCallback {
-
-		@Override
-		public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception ex) {
-			commitInProgress = false;
-
-			if (ex != null) {
-				log.warn("Committing offsets to Kafka failed. This does not compromise Flink's checkpoints.", ex);
-			}
-		}
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java
deleted file mode 100644
index 7a82365..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java
+++ /dev/null
@@ -1,482 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.core.testutils.MultiShotLatch;
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.internal.Handover;
-import org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher;
-import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-
-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.clients.consumer.OffsetCommitCallback;
-import org.apache.kafka.common.TopicPartition;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import org.mockito.Mockito;
-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.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantLock;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.anyLong;
-import static org.powermock.api.mockito.PowerMockito.doAnswer;
-import static org.powermock.api.mockito.PowerMockito.mock;
-import static org.powermock.api.mockito.PowerMockito.when;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
-/**
- * Unit tests for the {@link Kafka09Fetcher}.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(KafkaConsumerThread.class)
-public class Kafka09FetcherTest {
-
-	@Test
-	public void testCommitDoesNotBlock() throws Exception {
-
-		// test data
-		final KafkaTopicPartition testPartition = new KafkaTopicPartition("test", 42);
-		final Map<KafkaTopicPartition, Long> testCommitData = new HashMap<>();
-		testCommitData.put(testPartition, 11L);
-
-		// to synchronize when the consumer is in its blocking method
-		final OneShotLatch sync = new OneShotLatch();
-
-		// ----- the mock consumer with blocking poll calls ----
-		final MultiShotLatch blockerLatch = new MultiShotLatch();
-		
-		KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
-		when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
-			
-			@Override
-			public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) throws InterruptedException {
-				sync.trigger();
-				blockerLatch.await();
-				return ConsumerRecords.empty();
-			}
-		});
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocation) {
-				blockerLatch.trigger();
-				return null;
-			}
-		}).when(mockConsumer).wakeup();
-
-		// make sure the fetcher creates the mock consumer
-		whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
-
-		// ----- create the test fetcher -----
-
-		@SuppressWarnings("unchecked")
-		SourceContext<String> sourceContext = mock(SourceContext.class);
-		List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition("test", 42));
-		KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
-
-		final Kafka09Fetcher<String> fetcher = new Kafka09Fetcher<>(
-				sourceContext,
-				topics,
-				null, /* periodic watermark extractor */
-				null, /* punctuated watermark extractor */
-				new TestProcessingTimeService(),
-				10, /* watermark interval */
-				this.getClass().getClassLoader(),
-				true, /* checkpointing */
-				"task_name",
-				new UnregisteredMetricsGroup(),
-				schema,
-				new Properties(),
-				0L,
-				false);
-
-		// ----- run the fetcher -----
-
-		final AtomicReference<Throwable> error = new AtomicReference<>();
-		final Thread fetcherRunner = new Thread("fetcher runner") {
-
-			@Override
-			public void run() {
-				try {
-					fetcher.runFetchLoop();
-				} catch (Throwable t) {
-					error.set(t);
-				}
-			}
-		};
-		fetcherRunner.start();
-
-		// wait until the fetcher has reached the method of interest
-		sync.await();
-
-		// ----- trigger the offset commit -----
-		
-		final AtomicReference<Throwable> commitError = new AtomicReference<>();
-		final Thread committer = new Thread("committer runner") {
-			@Override
-			public void run() {
-				try {
-					fetcher.commitInternalOffsetsToKafka(testCommitData);
-				} catch (Throwable t) {
-					commitError.set(t);
-				}
-			}
-		};
-		committer.start();
-
-		// ----- ensure that the committer finishes in time  -----
-		committer.join(30000);
-		assertFalse("The committer did not finish in time", committer.isAlive());
-
-		// ----- test done, wait till the fetcher is done for a clean shutdown -----
-		fetcher.cancel();
-		fetcherRunner.join();
-
-		// check that there were no errors in the fetcher
-		final Throwable fetcherError = error.get();
-		if (fetcherError != null && !(fetcherError instanceof Handover.ClosedException)) {
-			throw new Exception("Exception in the fetcher", fetcherError);
-		}
-		final Throwable committerError = commitError.get();
-		if (committerError != null) {
-			throw new Exception("Exception in the committer", committerError);
-		}
-	}
-
-	@Test
-	public void ensureOffsetsGetCommitted() throws Exception {
-		
-		// test data
-		final KafkaTopicPartition testPartition1 = new KafkaTopicPartition("test", 42);
-		final KafkaTopicPartition testPartition2 = new KafkaTopicPartition("another", 99);
-		
-		final Map<KafkaTopicPartition, Long> testCommitData1 = new HashMap<>();
-		testCommitData1.put(testPartition1, 11L);
-		testCommitData1.put(testPartition2, 18L);
-
-		final Map<KafkaTopicPartition, Long> testCommitData2 = new HashMap<>();
-		testCommitData2.put(testPartition1, 19L);
-		testCommitData2.put(testPartition2, 28L);
-
-		final BlockingQueue<Map<TopicPartition, OffsetAndMetadata>> commitStore = new LinkedBlockingQueue<>();
-
-
-		// ----- the mock consumer with poll(), wakeup(), and commit(A)sync calls ----
-
-		final MultiShotLatch blockerLatch = new MultiShotLatch();
-
-		KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
-
-		when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
-			@Override
-			public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) throws InterruptedException {
-				blockerLatch.await();
-				return ConsumerRecords.empty();
-			}
-		});
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocation) {
-				blockerLatch.trigger();
-				return null;
-			}
-		}).when(mockConsumer).wakeup();
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocation) {
-				@SuppressWarnings("unchecked")
-				Map<TopicPartition, OffsetAndMetadata> offsets = 
-						(Map<TopicPartition, OffsetAndMetadata>) invocation.getArguments()[0];
-
-				OffsetCommitCallback callback = (OffsetCommitCallback) invocation.getArguments()[1];
-
-				commitStore.add(offsets);
-				callback.onComplete(offsets, null);
-
-				return null; 
-			}
-		}).when(mockConsumer).commitAsync(
-				Mockito.<Map<TopicPartition, OffsetAndMetadata>>any(), any(OffsetCommitCallback.class));
-
-		// make sure the fetcher creates the mock consumer
-		whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
-
-		// ----- create the test fetcher -----
-
-		@SuppressWarnings("unchecked")
-		SourceContext<String> sourceContext = mock(SourceContext.class);
-		List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition("test", 42));
-		KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
-
-		final Kafka09Fetcher<String> fetcher = new Kafka09Fetcher<>(
-				sourceContext,
-				topics,
-				null, /* periodic watermark extractor */
-				null, /* punctuated watermark extractor */
-				new TestProcessingTimeService(),
-				10, /* watermark interval */
-				this.getClass().getClassLoader(),
-				true, /* checkpointing */
-				"task_name",
-				new UnregisteredMetricsGroup(),
-				schema,
-				new Properties(),
-				0L,
-				false);
-
-
-		// ----- run the fetcher -----
-
-		final AtomicReference<Throwable> error = new AtomicReference<>();
-		final Thread fetcherRunner = new Thread("fetcher runner") {
-
-			@Override
-			public void run() {
-				try {
-					fetcher.runFetchLoop();
-				} catch (Throwable t) {
-					error.set(t);
-				}
-			}
-		};
-		fetcherRunner.start();
-
-		// ----- trigger the first offset commit -----
-
-		fetcher.commitInternalOffsetsToKafka(testCommitData1);
-		Map<TopicPartition, OffsetAndMetadata> result1 = commitStore.take();
-
-		for (Entry<TopicPartition, OffsetAndMetadata> entry : result1.entrySet()) {
-			TopicPartition partition = entry.getKey();
-			if (partition.topic().equals("test")) {
-				assertEquals(42, partition.partition());
-				assertEquals(12L, entry.getValue().offset());
-			}
-			else if (partition.topic().equals("another")) {
-				assertEquals(99, partition.partition());
-				assertEquals(17L, entry.getValue().offset());
-			}
-		}
-
-		// ----- trigger the second offset commit -----
-
-		fetcher.commitInternalOffsetsToKafka(testCommitData2);
-		Map<TopicPartition, OffsetAndMetadata> result2 = commitStore.take();
-
-		for (Entry<TopicPartition, OffsetAndMetadata> entry : result2.entrySet()) {
-			TopicPartition partition = entry.getKey();
-			if (partition.topic().equals("test")) {
-				assertEquals(42, partition.partition());
-				assertEquals(20L, entry.getValue().offset());
-			}
-			else if (partition.topic().equals("another")) {
-				assertEquals(99, partition.partition());
-				assertEquals(27L, entry.getValue().offset());
-			}
-		}
-		
-		// ----- test done, wait till the fetcher is done for a clean shutdown -----
-		fetcher.cancel();
-		fetcherRunner.join();
-
-		// check that there were no errors in the fetcher
-		final Throwable caughtError = error.get();
-		if (caughtError != null && !(caughtError instanceof Handover.ClosedException)) {
-			throw new Exception("Exception in the fetcher", caughtError);
-		}
-	}
-
-	@Test
-	public void testCancellationWhenEmitBlocks() throws Exception {
-
-		// ----- some test data -----
-
-		final String topic = "test-topic";
-		final int partition = 3;
-		final byte[] payload = new byte[] {1, 2, 3, 4};
-
-		final List<ConsumerRecord<byte[], byte[]>> records = Arrays.asList(
-				new ConsumerRecord<byte[], byte[]>(topic, partition, 15, payload, payload),
-				new ConsumerRecord<byte[], byte[]>(topic, partition, 16, payload, payload),
-				new ConsumerRecord<byte[], byte[]>(topic, partition, 17, payload, payload));
-
-		final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> data = new HashMap<>();
-		data.put(new TopicPartition(topic, partition), records);
-
-		final ConsumerRecords<byte[], byte[]> consumerRecords = new ConsumerRecords<>(data);
-
-		// ----- the test consumer -----
-
-		final KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
-		when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
-			@Override
-			public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) {
-				return consumerRecords;
-			}
-		});
-
-		whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
-
-		// ----- build a fetcher -----
-
-		BlockingSourceContext<String> sourceContext = new BlockingSourceContext<>();
-		List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition(topic, partition));
-		KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
-
-		final Kafka09Fetcher<String> fetcher = new Kafka09Fetcher<>(
-				sourceContext,
-				topics,
-				null, /* periodic watermark extractor */
-				null, /* punctuated watermark extractor */
-				new TestProcessingTimeService(),
-				10, /* watermark interval */
-				this.getClass().getClassLoader(),
-				true, /* checkpointing */
-				"task_name",
-				new UnregisteredMetricsGroup(),
-				schema,
-				new Properties(),
-				0L,
-				false);
-
-
-		// ----- run the fetcher -----
-
-		final AtomicReference<Throwable> error = new AtomicReference<>();
-		final Thread fetcherRunner = new Thread("fetcher runner") {
-
-			@Override
-			public void run() {
-				try {
-					fetcher.runFetchLoop();
-				} catch (Throwable t) {
-					error.set(t);
-				}
-			}
-		};
-		fetcherRunner.start();
-
-		// wait until the thread started to emit records to the source context
-		sourceContext.waitTillHasBlocker();
-
-		// now we try to cancel the fetcher, including the interruption usually done on the task thread
-		// once it has finished, there must be no more thread blocked on the source context
-		fetcher.cancel();
-		fetcherRunner.interrupt();
-		fetcherRunner.join();
-
-		assertFalse("fetcher threads did not properly finish", sourceContext.isStillBlocking());
-	}
-
-	// ------------------------------------------------------------------------
-	//  test utilities
-	// ------------------------------------------------------------------------
-
-	private static final class BlockingSourceContext<T> implements SourceContext<T> {
-
-		private final ReentrantLock lock = new ReentrantLock();
-		private final OneShotLatch inBlocking = new OneShotLatch();
-
-		@Override
-		public void collect(T element) {
-			block();
-		}
-
-		@Override
-		public void collectWithTimestamp(T element, long timestamp) {
-			block();
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			block();
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return new Object();
-		}
-
-		@Override
-		public void close() {}
-
-		public void waitTillHasBlocker() throws InterruptedException {
-			inBlocking.await();
-		}
-
-		public boolean isStillBlocking() {
-			return lock.isLocked();
-		}
-
-		@SuppressWarnings({"InfiniteLoopStatement", "SynchronizationOnLocalVariableOrMethodParameter"})
-		private void block() {
-			lock.lock();
-			try {
-				inBlocking.trigger();
-
-				// put this thread to sleep indefinitely
-				final Object o = new Object();
-				while (true) {
-					synchronized (o) {
-						o.wait();
-					}
-				}
-			}
-			catch (InterruptedException e) {
-				// exit cleanly, simply reset the interruption flag
-				Thread.currentThread().interrupt();
-			}
-			finally {
-				lock.unlock();
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index d18e2a9..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.junit.Test;
-
-public class Kafka09ITCase extends KafkaConsumerTestBase {
-
-	// ------------------------------------------------------------------------
-	//  Suite of Tests
-	// ------------------------------------------------------------------------
-
-	@Test(timeout = 60000)
-	public void testFailOnNoBroker() throws Exception {
-		runFailOnNoBrokerTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testConcurrentProducerConsumerTopology() throws Exception {
-		runSimpleConcurrentProducerConsumerTopology();
-	}
-
-
-	@Test(timeout = 60000)
-	public void testKeyValueSupport() throws Exception {
-		runKeyValueTest();
-	}
-
-	// --- canceling / failures ---
-
-	@Test(timeout = 60000)
-	public void testCancelingEmptyTopic() throws Exception {
-		runCancelingOnEmptyInputTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testCancelingFullTopic() throws Exception {
-		runCancelingOnFullInputTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testFailOnDeploy() throws Exception {
-		runFailOnDeployTest();
-	}
-
-
-	// --- source to partition mappings and exactly once ---
-
-	@Test(timeout = 60000)
-	public void testOneToOneSources() throws Exception {
-		runOneToOneExactlyOnceTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testOneSourceMultiplePartitions() throws Exception {
-		runOneSourceMultiplePartitionsExactlyOnceTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testMultipleSourcesOnePartition() throws Exception {
-		runMultipleSourcesOnePartitionExactlyOnceTest();
-	}
-
-	// --- broker failure ---
-
-	@Test(timeout = 60000)
-	public void testBrokerFailure() throws Exception {
-		runBrokerFailureTest();
-	}
-
-	// --- special executions ---
-
-	@Test(timeout = 60000)
-	public void testBigRecordJob() throws Exception {
-		runBigRecordTestTopology();
-	}
-
-	@Test(timeout = 60000)
-	public void testMultipleTopics() throws Exception {
-		runProduceConsumeMultipleTopics();
-	}
-
-	@Test(timeout = 60000)
-	public void testAllDeletes() throws Exception {
-		runAllDeletesTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testEndOfStream() throws Exception {
-		runEndOfStreamTest();
-	}
-
-	@Test(timeout = 60000)
-	public void testMetrics() throws Throwable {
-		runMetricsTest();
-	}
-
-	// --- offset committing ---
-
-	@Test(timeout = 60000)
-	public void testCommitOffsetsToKafka() throws Exception {
-		runCommitOffsetsToKafka();
-	}
-
-	@Test(timeout = 60000)
-	public void testStartFromKafkaCommitOffsets() throws Exception {
-		runStartFromKafkaCommitOffsets();
-	}
-
-	@Test(timeout = 60000)
-	public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception {
-		runAutoOffsetRetrievalAndCommitToKafka();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
deleted file mode 100644
index 45f70ac..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-import java.util.Properties;
-
-public class Kafka09JsonTableSinkTest extends KafkaTableSinkTestBase {
-
-	@Override
-	protected KafkaTableSink createTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner,
-			final FlinkKafkaProducerBase<Row> kafkaProducer) {
-
-		return new Kafka09JsonTableSink(topic, properties, partitioner) {
-			@Override
-			protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties,
-					SerializationSchema<Row> serializationSchema, KafkaPartitioner<Row> partitioner) {
-				return kafkaProducer;
-			}
-		};
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	protected SerializationSchema<Row> getSerializationSchema() {
-		return new JsonRowSerializationSchema(FIELD_NAMES);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
deleted file mode 100644
index 4a75f50..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import java.util.Properties;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
-
-public class Kafka09JsonTableSourceTest extends KafkaTableSourceTestBase {
-
-	@Override
-	protected KafkaTableSource createTableSource(String topic, Properties properties, String[] fieldNames, TypeInformation<?>[] typeInfo) {
-		return new Kafka09JsonTableSource(topic, properties, fieldNames, typeInfo);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	protected Class<DeserializationSchema<Row>> getDeserializationSchema() {
-		return (Class) JsonRowDeserializationSchema.class;
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	protected Class<FlinkKafkaConsumerBase<Row>> getFlinkKafkaConsumer() {
-		return (Class) FlinkKafkaConsumer09.class;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java
deleted file mode 100644
index ae4f5b2..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-
-import org.junit.Test;
-
-@SuppressWarnings("serial")
-public class Kafka09ProducerITCase extends KafkaProducerTestBase {
-
-	@Test
-	public void testCustomPartitioning() {
-		runCustomPartitioningTest();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java
deleted file mode 100644
index e748537..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.test.util.SecureTestEnvironment;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/*
- * Kafka Secure Connection (kerberos) IT test case
- */
-public class Kafka09SecuredRunITCase extends KafkaConsumerTestBase {
-
-	protected static final Logger LOG = LoggerFactory.getLogger(Kafka09SecuredRunITCase.class);
-
-	@BeforeClass
-	public static void prepare() throws IOException, ClassNotFoundException {
-		LOG.info("-------------------------------------------------------------------------");
-		LOG.info("    Starting Kafka09SecuredRunITCase ");
-		LOG.info("-------------------------------------------------------------------------");
-
-		SecureTestEnvironment.prepare(tempFolder);
-		SecureTestEnvironment.populateFlinkSecureConfigurations(getFlinkConfiguration());
-
-		startClusters(true);
-	}
-
-	@AfterClass
-	public static void shutDownServices() {
-		shutdownClusters();
-		SecureTestEnvironment.cleanup();
-	}
-
-
-	//timeout interval is large since in Travis, ZK connection timeout occurs frequently
-	//The timeout for the test case is 2 times timeout of ZK connection
-	@Test(timeout = 600000)
-	public void testMultipleTopics() throws Exception {
-		runProduceConsumeMultipleTopics();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index 18b2aec..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.operators.StreamSink;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
-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.Collections;
-import java.util.concurrent.Future;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(FlinkKafkaProducerBase.class)
-public class KafkaProducerTest extends TestLogger {
-	
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testPropagateExceptions() {
-		try {
-			// mock kafka producer
-			KafkaProducer<?, ?> kafkaProducerMock = mock(KafkaProducer.class);
-			
-			// partition setup
-			when(kafkaProducerMock.partitionsFor(anyString())).thenReturn(
-				// returning a unmodifiable list to mimic KafkaProducer#partitionsFor() behaviour
-				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)))
-				.thenAnswer(new Answer<Future<RecordMetadata>>() {
-					@Override
-					public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
-						Callback callback = (Callback) invocation.getArguments()[1];
-						callback.onCompletion(null, new Exception("Test error"));
-						return null;
-					}
-				});
-			
-			// make sure the FlinkKafkaProducer instantiates our mock producer
-			whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock);
-			
-			// (1) producer that propagates errors
-
-			FlinkKafkaProducer09<String> producerPropagating = new FlinkKafkaProducer09<>(
-					"mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), null);
-
-			OneInputStreamOperatorTestHarness<String, Object> testHarness =
-					new OneInputStreamOperatorTestHarness<>(new StreamSink(producerPropagating));
-
-			testHarness.open();
-
-			try {
-				testHarness.processElement(new StreamRecord<>("value"));
-				testHarness.processElement(new StreamRecord<>("value"));
-				fail("This should fail with an exception");
-			}
-			catch (Exception e) {
-				assertNotNull(e.getCause());
-				assertNotNull(e.getCause().getMessage());
-				assertTrue(e.getCause().getMessage().contains("Test error"));
-			}
-
-			// (2) producer that only logs errors
-
-			FlinkKafkaProducer09<String> producerLogging = new FlinkKafkaProducer09<>(
-					"mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), null);
-			producerLogging.setLogFailuresOnly(true);
-
-			testHarness = new OneInputStreamOperatorTestHarness<>(new StreamSink(producerLogging));
-
-			testHarness.open();
-
-			testHarness.processElement(new StreamRecord<>("value"));
-			testHarness.processElement(new StreamRecord<>("value"));
-
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
deleted file mode 100644
index 1802e0c..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
+++ /dev/null
@@ -1,439 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import kafka.admin.AdminUtils;
-import kafka.common.KafkaException;
-import kafka.api.PartitionMetadata;
-import kafka.network.SocketServer;
-import kafka.server.KafkaConfig;
-import kafka.server.KafkaServer;
-import kafka.utils.SystemTime$;
-import kafka.utils.ZkUtils;
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.commons.io.FileUtils;
-import org.apache.curator.test.TestingServer;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.util.NetUtils;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.protocol.SecurityProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.collection.Seq;
-
-import java.io.File;
-import java.net.BindException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.UUID;
-
-import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * An implementation of the KafkaServerProvider for Kafka 0.9
- */
-public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment {
-
-	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class);
-	private File tmpZkDir;
-	private File tmpKafkaParent;
-	private List<File> tmpKafkaDirs;
-	private List<KafkaServer> brokers;
-	private TestingServer zookeeper;
-	private String zookeeperConnectionString;
-	private String brokerConnectionString = "";
-	private Properties standardProps;
-	private Properties additionalServerProperties;
-	private boolean secureMode = false;
-	// 6 seconds is default. Seems to be too small for travis. 30 seconds
-	private String zkTimeout = "30000";
-
-	public String getBrokerConnectionString() {
-		return brokerConnectionString;
-	}
-
-	@Override
-	public Properties getStandardProperties() {
-		return standardProps;
-	}
-
-	@Override
-	public String getVersion() {
-		return "0.9";
-	}
-
-	@Override
-	public List<KafkaServer> getBrokers() {
-		return brokers;
-	}
-
-	@Override
-	public <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, KeyedDeserializationSchema<T> readSchema, Properties props) {
-		return new FlinkKafkaConsumer09<>(topics, readSchema, props);
-	}
-
-	@Override
-	public <T> StreamSink<T> getProducerSink(
-			String topic,
-			KeyedSerializationSchema<T> serSchema,
-			Properties props,
-			KafkaPartitioner<T> partitioner) {
-		FlinkKafkaProducer09<T> prod = new FlinkKafkaProducer09<>(topic, serSchema, props, partitioner);
-		prod.setFlushOnCheckpoint(true);
-		return new StreamSink<>(prod);
-	}
-
-	@Override
-	public <T> DataStreamSink<T> produceIntoKafka(DataStream<T> stream, String topic, KeyedSerializationSchema<T> serSchema, Properties props, KafkaPartitioner<T> partitioner) {
-		FlinkKafkaProducer09<T> prod = new FlinkKafkaProducer09<>(topic, serSchema, props, partitioner);
-		prod.setFlushOnCheckpoint(true);
-		return stream.addSink(prod);
-	}
-
-	@Override
-	public KafkaOffsetHandler createOffsetHandler(Properties props) {
-		return new KafkaOffsetHandlerImpl(props);
-	}
-
-	@Override
-	public void restartBroker(int leaderId) throws Exception {
-		brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId)));
-	}
-
-	@Override
-	public int getLeaderToShutDown(String topic) throws Exception {
-		ZkUtils zkUtils = getZkUtils();
-		try {
-			PartitionMetadata firstPart = null;
-			do {
-				if (firstPart != null) {
-					LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
-					// not the first try. Sleep a bit
-					Thread.sleep(150);
-				}
-
-				Seq<PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkUtils).partitionsMetadata();
-				firstPart = partitionMetadata.head();
-			}
-			while (firstPart.errorCode() != 0);
-
-			return firstPart.leader().get().id();
-		} finally {
-			zkUtils.close();
-		}
-	}
-
-	@Override
-	public int getBrokerId(KafkaServer server) {
-		return server.config().brokerId();
-	}
-
-	@Override
-	public boolean isSecureRunSupported() {
-		return true;
-	}
-
-	@Override
-	public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) {
-
-		//increase the timeout since in Travis ZK connection takes long time for secure connection.
-		if(secureMode) {
-			//run only one kafka server to avoid multiple ZK connections from many instances - Travis timeout
-			numKafkaServers = 1;
-			zkTimeout = String.valueOf(Integer.parseInt(zkTimeout) * 15);
-		}
-
-		this.additionalServerProperties = additionalServerProperties;
-		this.secureMode = secureMode;
-		File tempDir = new File(System.getProperty("java.io.tmpdir"));
-
-		tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
-		assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs());
-
-		tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString()));
-		assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs());
-
-		tmpKafkaDirs = new ArrayList<>(numKafkaServers);
-		for (int i = 0; i < numKafkaServers; i++) {
-			File tmpDir = new File(tmpKafkaParent, "server-" + i);
-			assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
-			tmpKafkaDirs.add(tmpDir);
-		}
-
-		zookeeper = null;
-		brokers = null;
-
-		try {
-			LOG.info("Starting Zookeeper");
-			zookeeper = new TestingServer(-1, tmpZkDir);
-			zookeeperConnectionString = zookeeper.getConnectString();
-			LOG.info("zookeeperConnectionString: {}", zookeeperConnectionString);
-
-			LOG.info("Starting KafkaServer");
-			brokers = new ArrayList<>(numKafkaServers);
-
-			for (int i = 0; i < numKafkaServers; i++) {
-				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i)));
-
-				SocketServer socketServer = brokers.get(i).socketServer();
-				if(secureMode) {
-					brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.SASL_PLAINTEXT)) + ",";
-				} else {
-					brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ",";
-				}
-			}
-
-			LOG.info("ZK and KafkaServer started.");
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("Test setup failed: " + t.getMessage());
-		}
-
-		LOG.info("brokerConnectionString --> {}", brokerConnectionString);
-
-		standardProps = new Properties();
-		standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
-		standardProps.setProperty("bootstrap.servers", brokerConnectionString);
-		standardProps.setProperty("group.id", "flink-tests");
-		standardProps.setProperty("enable.auto.commit", "false");
-		standardProps.setProperty("zookeeper.session.timeout.ms", zkTimeout);
-		standardProps.setProperty("zookeeper.connection.timeout.ms", zkTimeout);
-		standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning. (earliest is kafka 0.9 value)
-		standardProps.setProperty("max.partition.fetch.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
-
-	}
-
-	@Override
-	public void shutdown() {
-		for (KafkaServer broker : brokers) {
-			if (broker != null) {
-				broker.shutdown();
-			}
-		}
-		brokers.clear();
-
-		if (zookeeper != null) {
-			try {
-				zookeeper.stop();
-				zookeeper.close();
-			}
-			catch (Exception e) {
-				LOG.warn("ZK.stop() failed", e);
-			}
-			zookeeper = null;
-		}
-
-		// clean up the temp spaces
-
-		if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
-			try {
-				FileUtils.deleteDirectory(tmpKafkaParent);
-			}
-			catch (Exception e) {
-				// ignore
-			}
-		}
-		if (tmpZkDir != null && tmpZkDir.exists()) {
-			try {
-				FileUtils.deleteDirectory(tmpZkDir);
-			}
-			catch (Exception e) {
-				// ignore
-			}
-		}
-	}
-
-	public ZkUtils getZkUtils() {
-		LOG.info("In getZKUtils:: zookeeperConnectionString = {}", zookeeperConnectionString);
-		ZkClient creator = new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
-				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
-		return ZkUtils.apply(creator, false);
-	}
-
-	@Override
-	public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor, Properties topicConfig) {
-		// create topic with one client
-		LOG.info("Creating topic {}", topic);
-
-		ZkUtils zkUtils = getZkUtils();
-		try {
-			AdminUtils.createTopic(zkUtils, topic, numberOfPartitions, replicationFactor, topicConfig);
-		} finally {
-			zkUtils.close();
-		}
-
-		LOG.info("Topic {} create request is successfully posted", topic);
-
-		// validate that the topic has been created
-		final long deadline = System.currentTimeMillis() + Integer.parseInt(zkTimeout);
-		do {
-			try {
-				if(secureMode) {
-					//increase wait time since in Travis ZK timeout occurs frequently
-					int wait = Integer.parseInt(zkTimeout) / 100;
-					LOG.info("waiting for {} msecs before the topic {} can be checked", wait, topic);
-					Thread.sleep(wait);
-				} else {
-					Thread.sleep(100);
-				}
-
-			} catch (InterruptedException e) {
-				// restore interrupted state
-			}
-			// we could use AdminUtils.topicExists(zkUtils, topic) here, but it's results are
-			// not always correct.
-
-			LOG.info("Validating if the topic {} has been created or not", topic);
-
-			// create a new ZK utils connection
-			ZkUtils checkZKConn = getZkUtils();
-			if(AdminUtils.topicExists(checkZKConn, topic)) {
-				LOG.info("topic {} has been created successfully", topic);
-				checkZKConn.close();
-				return;
-			}
-			LOG.info("topic {} has not been created yet. Will check again...", topic);
-			checkZKConn.close();
-		}
-		while (System.currentTimeMillis() < deadline);
-		fail("Test topic could not be created");
-	}
-
-	@Override
-	public void deleteTestTopic(String topic) {
-		ZkUtils zkUtils = getZkUtils();
-		try {
-			LOG.info("Deleting topic {}", topic);
-
-			ZkClient zk = new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")),
-				Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer());
-
-			AdminUtils.deleteTopic(zkUtils, topic);
-
-			zk.close();
-		} finally {
-			zkUtils.close();
-		}
-	}
-
-	/**
-	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
-	 */
-	protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception {
-		Properties kafkaProperties = new Properties();
-
-		// properties have to be Strings
-		kafkaProperties.put("advertised.host.name", KAFKA_HOST);
-		kafkaProperties.put("broker.id", Integer.toString(brokerId));
-		kafkaProperties.put("log.dir", tmpFolder.toString());
-		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
-		kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
-		kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
-
-		// for CI stability, increase zookeeper session timeout
-		kafkaProperties.put("zookeeper.session.timeout.ms", zkTimeout);
-		kafkaProperties.put("zookeeper.connection.timeout.ms", zkTimeout);
-		if(additionalServerProperties != null) {
-			kafkaProperties.putAll(additionalServerProperties);
-		}
-
-		final int numTries = 5;
-
-		for (int i = 1; i <= numTries; i++) {
-			int kafkaPort = NetUtils.getAvailablePort();
-			kafkaProperties.put("port", Integer.toString(kafkaPort));
-
-			//to support secure kafka cluster
-			if(secureMode) {
-				LOG.info("Adding Kafka secure configurations");
-				kafkaProperties.put("listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort);
-				kafkaProperties.put("advertised.listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort);
-				kafkaProperties.putAll(getSecureProperties());
-			}
-
-			KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
-
-			try {
-				scala.Option<String> stringNone = scala.Option.apply(null);
-				KafkaServer server = new KafkaServer(kafkaConfig, SystemTime$.MODULE$, stringNone);
-				server.startup();
-				return server;
-			}
-			catch (KafkaException e) {
-				if (e.getCause() instanceof BindException) {
-					// port conflict, retry...
-					LOG.info("Port conflict when starting Kafka Broker. Retrying...");
-				}
-				else {
-					throw e;
-				}
-			}
-		}
-
-		throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
-	}
-
-	public Properties getSecureProperties() {
-		Properties prop = new Properties();
-		if(secureMode) {
-			prop.put("security.inter.broker.protocol", "SASL_PLAINTEXT");
-			prop.put("security.protocol", "SASL_PLAINTEXT");
-			prop.put("sasl.kerberos.service.name", "kafka");
-
-			//add special timeout for Travis
-			prop.setProperty("zookeeper.session.timeout.ms", zkTimeout);
-			prop.setProperty("zookeeper.connection.timeout.ms", zkTimeout);
-			prop.setProperty("metadata.fetch.timeout.ms","120000");
-		}
-		return prop;
-	}
-
-	private class KafkaOffsetHandlerImpl implements KafkaOffsetHandler {
-
-		private final KafkaConsumer<byte[], byte[]> offsetClient;
-
-		public KafkaOffsetHandlerImpl(Properties props) {
-			offsetClient = new KafkaConsumer<>(props);
-		}
-
-		@Override
-		public Long getCommittedOffset(String topicName, int partition) {
-			OffsetAndMetadata committed = offsetClient.committed(new TopicPartition(topicName, partition));
-			return (committed != null) ? committed.offset() : null;
-		}
-
-		@Override
-		public void close() {
-			offsetClient.close();
-		}
-	}
-
-}


[04/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index dccf698..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
+++ /dev/null
@@ -1,291 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-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.source.RichParallelSourceFunction;
-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.util.InstantiationUtil;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-import org.junit.ClassRule;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-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 LocalFlinkMiniCluster flink;
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	protected static Properties secureProps = new Properties();
-
-	@BeforeClass
-	public static void prepare() throws IOException, ClassNotFoundException {
-		LOG.info("-------------------------------------------------------------------------");
-		LOG.info("    Starting KafkaShortRetentionTestBase ");
-		LOG.info("-------------------------------------------------------------------------");
-
-		Configuration flinkConfig = new Configuration();
-
-		// dynamically load the implementation for the test
-		Class<?> clazz = Class.forName("org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl");
-		kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz);
-
-		LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion());
-
-		if(kafkaServer.isSecureRunSupported()) {
-			secureProps = kafkaServer.getSecureProperties();
-		}
-
-		Properties specificProperties = new Properties();
-		specificProperties.setProperty("log.retention.hours", "0");
-		specificProperties.setProperty("log.retention.minutes", "0");
-		specificProperties.setProperty("log.retention.ms", "250");
-		specificProperties.setProperty("log.retention.check.interval.ms", "100");
-		kafkaServer.prepare(1, specificProperties, false);
-
-		standardProps = kafkaServer.getStandardProperties();
-
-		// start also a re-usable Flink mini cluster
-		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
-		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
-		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
-		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
-
-		flink = new LocalFlinkMiniCluster(flinkConfig, false);
-		flink.start();
-	}
-
-	@AfterClass
-	public static void shutDownServices() {
-		if (flink != null) {
-			flink.shutdown();
-		}
-		kafkaServer.shutdown();
-
-		secureProps.clear();
-	}
-
-	/**
-	 * This test is concurrently reading and writing from a kafka topic.
-	 * The job will run for a while
-	 * In a special deserializationSchema, we make sure that the offsets from the topic
-	 * are non-continuous (because the data is expiring faster than its consumed --> with auto.offset.reset = 'earliest', some offsets will not show up)
-	 *
-	 */
-	private static boolean stopProducer = false;
-
-	public void runAutoOffsetResetTest() throws Exception {
-		final String topic = "auto-offset-reset-test";
-
-		final int parallelism = 1;
-		final int elementsPerPartition = 50000;
-
-		Properties tprops = new Properties();
-		tprops.setProperty("retention.ms", "250");
-		kafkaServer.createTestTopic(topic, parallelism, 1, tprops);
-
-		final StreamExecutionEnvironment env =
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flink.getLeaderRPCPort());
-		env.setParallelism(parallelism);
-		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
-		env.getConfig().disableSysoutLogging();
-
-
-		// ----------- add producer dataflow ----------
-
-
-		DataStream<String> stream = env.addSource(new RichParallelSourceFunction<String>() {
-
-			private boolean running = true;
-
-			@Override
-			public void run(SourceContext<String> ctx) throws InterruptedException {
-				int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition;
-				int limit = cnt + elementsPerPartition;
-
-
-				while (running && !stopProducer && cnt < limit) {
-					ctx.collect("element-" + cnt);
-					cnt++;
-					Thread.sleep(10);
-				}
-				LOG.info("Stopping producer");
-			}
-
-			@Override
-			public void cancel() {
-				running = false;
-			}
-		});
-		Properties props = new Properties();
-		props.putAll(standardProps);
-		props.putAll(secureProps);
-		kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), props, null);
-
-		// ----------- add consumer dataflow ----------
-
-		NonContinousOffsetsDeserializationSchema deserSchema = new NonContinousOffsetsDeserializationSchema();
-		FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer(topic, deserSchema, props);
-
-		DataStreamSource<String> consuming = env.addSource(source);
-		consuming.addSink(new DiscardingSink<String>());
-
-		tryExecute(env, "run auto offset reset test");
-
-		kafkaServer.deleteTestTopic(topic);
-	}
-
-	
-	private class NonContinousOffsetsDeserializationSchema implements KeyedDeserializationSchema<String> {
-		private int numJumps;
-		long nextExpected = 0;
-
-		@Override
-		public String deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
-			if(offset != nextExpected) {
-				numJumps++;
-				nextExpected = offset;
-				LOG.info("Registered now jump at offset {}", offset);
-			}
-			nextExpected++;
-			try {
-				Thread.sleep(10); // slow down data consumption to trigger log eviction
-			} catch (InterruptedException e) {
-				throw new RuntimeException("Stopping it");
-			}
-			return "";
-		}
-
-		@Override
-		public boolean isEndOfStream(String nextElement) {
-			if( numJumps >= 5) {
-				// we saw 5 jumps and no failures --> consumer can handle auto.offset.reset
-				stopProducer = true;
-				return true;
-			}
-			return false;
-		}
-
-		@Override
-		public TypeInformation<String> getProducedType() {
-			return TypeInfoParser.parse("String");
-		}
-	}
-
-
-	/**
-	 * Ensure that the consumer is properly failing if "auto.offset.reset" is set to "none"
-	 * @throws Exception
-	 */
-	public void runFailOnAutoOffsetResetNone() throws Exception {
-		final String topic = "auto-offset-reset-none-test";
-		final int parallelism = 1;
-		
-		kafkaServer.createTestTopic(topic, parallelism, 1);
-
-		final StreamExecutionEnvironment env =
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flink.getLeaderRPCPort());
-		env.setParallelism(parallelism);
-		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
-		env.getConfig().disableSysoutLogging();
-		
-		// ----------- add consumer ----------
-
-		Properties customProps = new Properties();
-		customProps.putAll(standardProps);
-		customProps.putAll(secureProps);
-		customProps.setProperty("auto.offset.reset", "none"); // test that "none" leads to an exception
-		FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), customProps);
-
-		DataStreamSource<String> consuming = env.addSource(source);
-		consuming.addSink(new DiscardingSink<String>());
-
-		try {
-			env.execute("Test auto offset reset none");
-		} catch(Throwable e) {
-			System.out.println("MESSAGE: " + e.getCause().getCause().getMessage());
-			// check if correct exception has been thrown
-			if(!e.getCause().getCause().getMessage().contains("Unable to find previous offset")  // kafka 0.8
-			 && !e.getCause().getCause().getMessage().contains("Undefined offset with no reset policy for partition") // kafka 0.9
-					) {
-				throw e;
-			}
-		}
-
-		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.putAll(secureProps);
-		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/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
deleted file mode 100644
index ae0af52..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.typeutils.RowTypeInfo;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.connectors.kafka.internals.TypeUtil;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.junit.Test;
-
-import java.io.Serializable;
-import java.util.Properties;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotSame;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-
-public abstract class KafkaTableSinkTestBase {
-
-	private static final String TOPIC = "testTopic";
-	protected static final String[] FIELD_NAMES = new String[] {"field1", "field2"};
-	private static final TypeInformation[] FIELD_TYPES = TypeUtil.toTypeInfo(new Class[] {Integer.class, String.class});
-	private static final KafkaPartitioner<Row> PARTITIONER = new CustomPartitioner();
-	private static final Properties PROPERTIES = createSinkProperties();
-	@SuppressWarnings("unchecked")
-	private final FlinkKafkaProducerBase<Row> PRODUCER = new FlinkKafkaProducerBase<Row>(
-		TOPIC, new KeyedSerializationSchemaWrapper(getSerializationSchema()), PROPERTIES, PARTITIONER) {
-
-		@Override
-		protected void flush() {}
-	};
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testKafkaTableSink() throws Exception {
-		DataStream dataStream = mock(DataStream.class);
-
-		KafkaTableSink kafkaTableSink = spy(createTableSink());
-		kafkaTableSink.emitDataStream(dataStream);
-
-		verify(dataStream).addSink(eq(PRODUCER));
-
-		verify(kafkaTableSink).createKafkaProducer(
-			eq(TOPIC),
-			eq(PROPERTIES),
-			any(getSerializationSchema().getClass()),
-			eq(PARTITIONER));
-	}
-
-	@Test
-	public void testConfiguration() {
-		KafkaTableSink kafkaTableSink = createTableSink();
-		KafkaTableSink newKafkaTableSink = kafkaTableSink.configure(FIELD_NAMES, FIELD_TYPES);
-		assertNotSame(kafkaTableSink, newKafkaTableSink);
-
-		assertArrayEquals(FIELD_NAMES, newKafkaTableSink.getFieldNames());
-		assertArrayEquals(FIELD_TYPES, newKafkaTableSink.getFieldTypes());
-		assertEquals(new RowTypeInfo(FIELD_TYPES), newKafkaTableSink.getOutputType());
-	}
-
-	protected abstract KafkaTableSink createTableSink(String topic, Properties properties,
-			KafkaPartitioner<Row> partitioner, FlinkKafkaProducerBase<Row> kafkaProducer);
-
-	protected abstract SerializationSchema<Row> getSerializationSchema();
-
-	private KafkaTableSink createTableSink() {
-		return createTableSink(TOPIC, PROPERTIES, PARTITIONER, PRODUCER);
-	}
-
-	private static Properties createSinkProperties() {
-		Properties properties = new Properties();
-		properties.setProperty("bootstrap.servers", "localhost:12345");
-		return properties;
-	}
-
-	private static class CustomPartitioner extends KafkaPartitioner<Row> implements Serializable {
-		@Override
-		public int partition(Row next, byte[] serializedKey, byte[] serializedValue, int numPartitions) {
-			return 0;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
deleted file mode 100644
index 2a281e8..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import java.util.Properties;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.junit.Test;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-
-public abstract class KafkaTableSourceTestBase {
-
-	private static final String TOPIC = "testTopic";
-	private static final String[] FIELD_NAMES = new String[] { "long", "string", "boolean", "double", "missing-field" };
-	private static final TypeInformation<?>[] FIELD_TYPES = new TypeInformation<?>[] {
-		BasicTypeInfo.LONG_TYPE_INFO,
-		BasicTypeInfo.STRING_TYPE_INFO,
-		BasicTypeInfo.BOOLEAN_TYPE_INFO,
-		BasicTypeInfo.DOUBLE_TYPE_INFO,
-		BasicTypeInfo.LONG_TYPE_INFO };
-	private static final Properties PROPERTIES = createSourceProperties();
-
-	@Test
-	public void testKafkaTableSource() {
-		KafkaTableSource kafkaTableSource = spy(createTableSource());
-		StreamExecutionEnvironment env = mock(StreamExecutionEnvironment.class);
-		kafkaTableSource.getDataStream(env);
-
-		verify(env).addSource(any(getFlinkKafkaConsumer()));
-
-		verify(kafkaTableSource).getKafkaConsumer(
-			eq(TOPIC),
-			eq(PROPERTIES),
-			any(getDeserializationSchema()));
-	}
-
-	protected abstract KafkaTableSource createTableSource(String topic, Properties properties,
-			String[] fieldNames, TypeInformation<?>[] typeInfo);
-
-	protected abstract Class<DeserializationSchema<Row>> getDeserializationSchema();
-
-	protected abstract Class<FlinkKafkaConsumerBase<Row>> getFlinkKafkaConsumer();
-
-	private KafkaTableSource createTableSource() {
-		return createTableSource(TOPIC, PROPERTIES, FIELD_NAMES, FIELD_TYPES);
-	}
-
-	private static Properties createSourceProperties() {
-		Properties properties = new Properties();
-		properties.setProperty("zookeeper.connect", "dummy");
-		properties.setProperty("group.id", "dummy");
-		return properties;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index 5cec4f0..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.metrics.jmx.JMXReporter;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-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;
-
-import org.junit.ClassRule;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.IOException;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-
-
-/**
- * The base for the Kafka tests. It brings up:
- * <ul>
- *     <li>A ZooKeeper mini cluster</li>
- *     <li>Three Kafka Brokers (mini clusters)</li>
- *     <li>A Flink mini cluster</li>
- * </ul>
- * 
- * <p>Code in this test is based on the following GitHub repository:
- * <a href="https://github.com/sakserv/hadoop-mini-clusters">
- *   https://github.com/sakserv/hadoop-mini-clusters</a> (ASL licensed),
- * as per commit <i>bc6b2b2d5f6424d5f377aa6c0871e82a956462ef</i></p>
- */
-@SuppressWarnings("serial")
-public abstract class KafkaTestBase extends TestLogger {
-
-	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class);
-	
-	protected static final int NUMBER_OF_KAFKA_SERVERS = 3;
-
-	protected static String brokerConnectionStrings;
-
-	protected static Properties standardProps;
-	
-	protected static LocalFlinkMiniCluster flink;
-
-	protected static int flinkPort;
-
-	protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
-
-	protected static KafkaTestEnvironment kafkaServer;
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	protected static Properties secureProps = new Properties();
-
-	// ------------------------------------------------------------------------
-	//  Setup and teardown of the mini clusters
-	// ------------------------------------------------------------------------
-	
-	@BeforeClass
-	public static void prepare() throws IOException, ClassNotFoundException {
-
-		LOG.info("-------------------------------------------------------------------------");
-		LOG.info("    Starting KafkaTestBase ");
-		LOG.info("-------------------------------------------------------------------------");
-
-		startClusters(false);
-
-	}
-
-	@AfterClass
-	public static void shutDownServices() {
-
-		LOG.info("-------------------------------------------------------------------------");
-		LOG.info("    Shut down KafkaTestBase ");
-		LOG.info("-------------------------------------------------------------------------");
-
-		shutdownClusters();
-
-		LOG.info("-------------------------------------------------------------------------");
-		LOG.info("    KafkaTestBase finished");
-		LOG.info("-------------------------------------------------------------------------");
-	}
-
-	protected static Configuration getFlinkConfiguration() {
-		Configuration flinkConfig = new Configuration();
-		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
-		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
-		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
-		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
-		flinkConfig.setString(ConfigConstants.METRICS_REPORTERS_LIST, "my_reporter");
-		flinkConfig.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "my_reporter." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName());
-		return flinkConfig;
-	}
-
-	protected static void startClusters(boolean secureMode) throws ClassNotFoundException {
-
-		// dynamically load the implementation for the test
-		Class<?> clazz = Class.forName("org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl");
-		kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz);
-
-		LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion());
-
-		kafkaServer.prepare(NUMBER_OF_KAFKA_SERVERS, secureMode);
-
-		standardProps = kafkaServer.getStandardProperties();
-
-		brokerConnectionStrings = kafkaServer.getBrokerConnectionString();
-
-		if (secureMode) {
-			if (!kafkaServer.isSecureRunSupported()) {
-				throw new IllegalStateException(
-					"Attempting to test in secure mode but secure mode not supported by the KafkaTestEnvironment.");
-			}
-			secureProps = kafkaServer.getSecureProperties();
-		}
-
-		// start also a re-usable Flink mini cluster
-		flink = new LocalFlinkMiniCluster(getFlinkConfiguration(), false);
-		flink.start();
-
-		flinkPort = flink.getLeaderRPCPort();
-
-	}
-
-	protected static void shutdownClusters() {
-
-		flinkPort = -1;
-		if (flink != null) {
-			flink.shutdown();
-		}
-
-		if(secureProps != null) {
-			secureProps.clear();
-		}
-
-		kafkaServer.shutdown();
-
-	}
-
-
-
-	// ------------------------------------------------------------------------
-	//  Execution utilities
-	// ------------------------------------------------------------------------
-	
-
-	protected static void tryExecutePropagateExceptions(StreamExecutionEnvironment see, String name) throws Exception {
-		try {
-			see.execute(name);
-		}
-		catch (ProgramInvocationException | JobExecutionException root) {
-			Throwable cause = root.getCause();
-
-			// search for nested SuccessExceptions
-			int depth = 0;
-			while (!(cause instanceof SuccessException)) {
-				if (cause == null || depth++ == 20) {
-					throw root;
-				}
-				else {
-					cause = cause.getCause();
-				}
-			}
-		}
-	}
-
-	protected static void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
-		kafkaServer.createTestTopic(topic, numberOfPartitions, replicationFactor);
-	}
-	
-	protected static void deleteTestTopic(String topic) {
-		kafkaServer.deleteTestTopic(topic);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
deleted file mode 100644
index 10c7b86..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import kafka.server.KafkaServer;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-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.streaming.util.serialization.KeyedSerializationSchema;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.Properties;
-
-/**
- * Abstract class providing a Kafka test environment
- */
-public abstract class KafkaTestEnvironment {
-
-	protected static final String KAFKA_HOST = "localhost";
-
-	public abstract void prepare(int numKafkaServers, Properties kafkaServerProperties, boolean secureMode);
-
-	public void prepare(int numberOfKafkaServers, boolean secureMode) {
-		this.prepare(numberOfKafkaServers, null, secureMode);
-	}
-
-	public abstract void shutdown();
-
-	public abstract void deleteTestTopic(String topic);
-
-	public abstract void createTestTopic(String topic, int numberOfPartitions, int replicationFactor, Properties properties);
-
-	public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
-		this.createTestTopic(topic, numberOfPartitions, replicationFactor, new Properties());
-	}
-
-	public abstract Properties getStandardProperties();
-
-	public abstract Properties getSecureProperties();
-
-	public abstract String getBrokerConnectionString();
-
-	public abstract String getVersion();
-
-	public abstract List<KafkaServer> getBrokers();
-
-	// -- consumer / producer instances:
-	public <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, DeserializationSchema<T> deserializationSchema, Properties props) {
-		return getConsumer(topics, new KeyedDeserializationSchemaWrapper<T>(deserializationSchema), props);
-	}
-
-	public <T> FlinkKafkaConsumerBase<T> getConsumer(String topic, KeyedDeserializationSchema<T> readSchema, Properties props) {
-		return getConsumer(Collections.singletonList(topic), readSchema, props);
-	}
-
-	public <T> FlinkKafkaConsumerBase<T> getConsumer(String topic, DeserializationSchema<T> deserializationSchema, Properties props) {
-		return getConsumer(Collections.singletonList(topic), deserializationSchema, props);
-	}
-
-	public abstract <T> FlinkKafkaConsumerBase<T> getConsumer(List<String> topics, KeyedDeserializationSchema<T> readSchema, Properties props);
-
-	public abstract <T> StreamSink<T> getProducerSink(String topic,
-			KeyedSerializationSchema<T> serSchema, Properties props,
-			KafkaPartitioner<T> partitioner);
-
-	public abstract <T> DataStreamSink<T> produceIntoKafka(DataStream<T> stream, String topic,
-														KeyedSerializationSchema<T> serSchema, Properties props,
-														KafkaPartitioner<T> partitioner);
-
-	// -- offset handlers
-
-	public interface KafkaOffsetHandler {
-		Long getCommittedOffset(String topicName, int partition);
-		void close();
-	}
-
-	public abstract KafkaOffsetHandler createOffsetHandler(Properties props);
-
-	// -- leader failure simulation
-
-	public abstract void restartBroker(int leaderId) throws Exception;
-
-	public abstract int getLeaderToShutDown(String topic) throws Exception;
-
-	public abstract int getBrokerId(KafkaServer server);
-
-	public abstract boolean isSecureRunSupported();
-
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
deleted file mode 100644
index 0b3507a..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
+++ /dev/null
@@ -1,320 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.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.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.util.SerializedValue;
-
-import org.junit.Test;
-
-import javax.annotation.Nullable;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.*;
-
-@SuppressWarnings("serial")
-public class AbstractFetcherTimestampsTest {
-	
-	@Test
-	public void testPunctuatedWatermarks() throws Exception {
-		final String testTopic = "test topic name";
-		List<KafkaTopicPartition> originalPartitions = Arrays.asList(
-				new KafkaTopicPartition(testTopic, 7),
-				new KafkaTopicPartition(testTopic, 13),
-				new KafkaTopicPartition(testTopic, 21));
-
-		TestSourceContext<Long> sourceContext = new TestSourceContext<>();
-
-		TestProcessingTimeService processingTimeProvider = new TestProcessingTimeService();
-
-		TestFetcher<Long> fetcher = new TestFetcher<>(
-				sourceContext,
-				originalPartitions,
-				null, /* periodic watermark assigner */
-				new SerializedValue<AssignerWithPunctuatedWatermarks<Long>>(new PunctuatedTestExtractor()),
-				processingTimeProvider,
-				0);
-
-		final KafkaTopicPartitionState<Object> part1 = fetcher.subscribedPartitions()[0];
-		final KafkaTopicPartitionState<Object> part2 = fetcher.subscribedPartitions()[1];
-		final KafkaTopicPartitionState<Object> part3 = fetcher.subscribedPartitions()[2];
-
-		// elements generate a watermark if the timestamp is a multiple of three
-		
-		// elements for partition 1
-		fetcher.emitRecord(1L, part1, 1L);
-		fetcher.emitRecord(2L, part1, 2L);
-		fetcher.emitRecord(3L, part1, 3L);
-		assertEquals(3L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(3L, sourceContext.getLatestElement().getTimestamp());
-		assertFalse(sourceContext.hasWatermark());
-
-		// elements for partition 2
-		fetcher.emitRecord(12L, part2, 1L);
-		assertEquals(12L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(12L, sourceContext.getLatestElement().getTimestamp());
-		assertFalse(sourceContext.hasWatermark());
-
-		// elements for partition 3
-		fetcher.emitRecord(101L, part3, 1L);
-		fetcher.emitRecord(102L, part3, 2L);
-		assertEquals(102L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(102L, sourceContext.getLatestElement().getTimestamp());
-		
-		// now, we should have a watermark
-		assertTrue(sourceContext.hasWatermark());
-		assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp());
-		
-		// advance partition 3
-		fetcher.emitRecord(1003L, part3, 3L);
-		fetcher.emitRecord(1004L, part3, 4L);
-		fetcher.emitRecord(1005L, part3, 5L);
-		assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(1005L, sourceContext.getLatestElement().getTimestamp());
-
-		// advance partition 1 beyond partition 2 - this bumps the watermark
-		fetcher.emitRecord(30L, part1, 4L);
-		assertEquals(30L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(30L, sourceContext.getLatestElement().getTimestamp());
-		assertTrue(sourceContext.hasWatermark());
-		assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp());
-
-		// advance partition 2 again - this bumps the watermark
-		fetcher.emitRecord(13L, part2, 2L);
-		assertFalse(sourceContext.hasWatermark());
-		fetcher.emitRecord(14L, part2, 3L);
-		assertFalse(sourceContext.hasWatermark());
-		fetcher.emitRecord(15L, part2, 3L);
-		assertTrue(sourceContext.hasWatermark());
-		assertEquals(15L, sourceContext.getLatestWatermark().getTimestamp());
-	}
-	
-	@Test
-	public void testPeriodicWatermarks() throws Exception {
-		final String testTopic = "test topic name";
-		List<KafkaTopicPartition> originalPartitions = Arrays.asList(
-				new KafkaTopicPartition(testTopic, 7),
-				new KafkaTopicPartition(testTopic, 13),
-				new KafkaTopicPartition(testTopic, 21));
-
-		TestSourceContext<Long> sourceContext = new TestSourceContext<>();
-
-		TestProcessingTimeService processingTimeService = new TestProcessingTimeService();
-
-		TestFetcher<Long> fetcher = new TestFetcher<>(
-				sourceContext,
-				originalPartitions,
-				new SerializedValue<AssignerWithPeriodicWatermarks<Long>>(new PeriodicTestExtractor()),
-				null, /* punctuated watermarks assigner*/
-				processingTimeService,
-				10);
-
-		final KafkaTopicPartitionState<Object> part1 = fetcher.subscribedPartitions()[0];
-		final KafkaTopicPartitionState<Object> part2 = fetcher.subscribedPartitions()[1];
-		final KafkaTopicPartitionState<Object> part3 = fetcher.subscribedPartitions()[2];
-
-		// elements generate a watermark if the timestamp is a multiple of three
-
-		// elements for partition 1
-		fetcher.emitRecord(1L, part1, 1L);
-		fetcher.emitRecord(2L, part1, 2L);
-		fetcher.emitRecord(3L, part1, 3L);
-		assertEquals(3L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(3L, sourceContext.getLatestElement().getTimestamp());
-
-		// elements for partition 2
-		fetcher.emitRecord(12L, part2, 1L);
-		assertEquals(12L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(12L, sourceContext.getLatestElement().getTimestamp());
-
-		// elements for partition 3
-		fetcher.emitRecord(101L, part3, 1L);
-		fetcher.emitRecord(102L, part3, 2L);
-		assertEquals(102L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(102L, sourceContext.getLatestElement().getTimestamp());
-
-		processingTimeService.setCurrentTime(10);
-
-		// now, we should have a watermark (this blocks until the periodic thread emitted the watermark)
-		assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp());
-
-		// advance partition 3
-		fetcher.emitRecord(1003L, part3, 3L);
-		fetcher.emitRecord(1004L, part3, 4L);
-		fetcher.emitRecord(1005L, part3, 5L);
-		assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(1005L, sourceContext.getLatestElement().getTimestamp());
-
-		// advance partition 1 beyond partition 2 - this bumps the watermark
-		fetcher.emitRecord(30L, part1, 4L);
-		assertEquals(30L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(30L, sourceContext.getLatestElement().getTimestamp());
-
-		processingTimeService.setCurrentTime(20);
-
-		// this blocks until the periodic thread emitted the watermark
-		assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp());
-
-		// advance partition 2 again - this bumps the watermark
-		fetcher.emitRecord(13L, part2, 2L);
-		fetcher.emitRecord(14L, part2, 3L);
-		fetcher.emitRecord(15L, part2, 3L);
-
-		processingTimeService.setCurrentTime(30);
-		// this blocks until the periodic thread emitted the watermark
-		long watermarkTs = sourceContext.getLatestWatermark().getTimestamp();
-		assertTrue(watermarkTs >= 13L && watermarkTs <= 15L);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Test mocks
-	// ------------------------------------------------------------------------
-
-	private static final class TestFetcher<T> extends AbstractFetcher<T, Object> {
-
-		protected TestFetcher(
-				SourceContext<T> sourceContext,
-				List<KafkaTopicPartition> assignedPartitions,
-				SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
-				SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
-				ProcessingTimeService processingTimeProvider,
-				long autoWatermarkInterval) throws Exception
-		{
-			super(sourceContext, assignedPartitions, watermarksPeriodic, watermarksPunctuated, processingTimeProvider, autoWatermarkInterval, TestFetcher.class.getClassLoader(), false);
-		}
-
-		@Override
-		public void runFetchLoop() throws Exception {
-			throw new UnsupportedOperationException();
-		}
-
-		@Override
-		public void cancel() {
-			throw new UnsupportedOperationException();
-		}
-
-		@Override
-		public Object createKafkaPartitionHandle(KafkaTopicPartition partition) {
-			return new Object();
-		}
-
-		@Override
-		public void commitInternalOffsetsToKafka(Map<KafkaTopicPartition, Long> offsets) throws Exception {
-			throw new UnsupportedOperationException();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static final class TestSourceContext<T> implements SourceContext<T> {
-
-		private final Object checkpointLock = new Object();
-		private final Object watermarkLock = new Object();
-
-		private volatile StreamRecord<T> latestElement;
-		private volatile Watermark currentWatermark;
-
-		@Override
-		public void collect(T element) {
-			throw new UnsupportedOperationException();
-		}
-
-		@Override
-		public void collectWithTimestamp(T element, long timestamp) {
-			this.latestElement = new StreamRecord<>(element, timestamp);
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			synchronized (watermarkLock) {
-				currentWatermark = mark;
-				watermarkLock.notifyAll();
-			}
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return checkpointLock;
-		}
-
-		@Override
-		public void close() {}
-
-		public StreamRecord<T> getLatestElement() {
-			return latestElement;
-		}
-
-		public boolean hasWatermark() {
-			return currentWatermark != null;
-		}
-		
-		public Watermark getLatestWatermark() throws InterruptedException {
-			synchronized (watermarkLock) {
-				while (currentWatermark == null) {
-					watermarkLock.wait();
-				}
-				Watermark wm = currentWatermark;
-				currentWatermark = null;
-				return wm;
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static class PeriodicTestExtractor implements AssignerWithPeriodicWatermarks<Long> {
-
-		private volatile long maxTimestamp = Long.MIN_VALUE;
-		
-		@Override
-		public long extractTimestamp(Long element, long previousElementTimestamp) {
-			maxTimestamp = Math.max(maxTimestamp, element);
-			return element;
-		}
-
-		@Nullable
-		@Override
-		public Watermark getCurrentWatermark() {
-			return new Watermark(maxTimestamp);
-		}
-	}
-
-	private static class PunctuatedTestExtractor implements AssignerWithPunctuatedWatermarks<Long> {
-
-		@Override
-		public long extractTimestamp(Long element, long previousElementTimestamp) {
-			return element;
-		}
-
-		@Nullable
-		@Override
-		public Watermark checkAndGetNextWatermark(Long lastElement, long extractedTimestamp) {
-			return extractedTimestamp % 3 == 0 ? new Watermark(extractedTimestamp) : null;
-		}
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/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
deleted file mode 100644
index 0e16263..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.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/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
deleted file mode 100644
index 9e8e1d9..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.api.transformations.StreamTransformation;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase;
-import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironment;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
-
-import java.util.Collection;
-import java.util.Properties;
-import java.util.Random;
-
-import static org.mockito.Mockito.mock;
-
-@SuppressWarnings("serial")
-public class DataGenerators {
-
-	public static void generateRandomizedIntegerSequence(StreamExecutionEnvironment env,
-														 KafkaTestEnvironment testServer, String topic,
-														 final int numPartitions,
-														 final int numElements,
-														 final boolean randomizeOrder) throws Exception {
-		env.setParallelism(numPartitions);
-		env.getConfig().disableSysoutLogging();
-		env.setRestartStrategy(RestartStrategies.noRestart());
-
-		DataStream<Integer> stream = env.addSource(
-				new RichParallelSourceFunction<Integer>() {
-
-					private volatile boolean running = true;
-
-					@Override
-					public void run(SourceContext<Integer> ctx) {
-						// create a sequence
-						int[] elements = new int[numElements];
-						for (int i = 0, val = getRuntimeContext().getIndexOfThisSubtask();
-							 i < numElements;
-							 i++, val += getRuntimeContext().getNumberOfParallelSubtasks()) {
-
-							elements[i] = val;
-						}
-
-						// scramble the sequence
-						if (randomizeOrder) {
-							Random rnd = new Random();
-							for (int i = 0; i < elements.length; i++) {
-								int otherPos = rnd.nextInt(elements.length);
-
-								int tmp = elements[i];
-								elements[i] = elements[otherPos];
-								elements[otherPos] = tmp;
-							}
-						}
-
-						// emit the sequence
-						int pos = 0;
-						while (running && pos < elements.length) {
-							ctx.collect(elements[pos++]);
-						}
-					}
-
-					@Override
-					public void cancel() {
-						running = false;
-					}
-				});
-
-		Properties props = new Properties();
-		props.putAll(FlinkKafkaProducerBase.getPropertiesFromBrokerList(testServer.getBrokerConnectionString()));
-		Properties secureProps = testServer.getSecureProperties();
-		if(secureProps != null) {
-			props.putAll(testServer.getSecureProperties());
-		}
-
-		stream = stream.rebalance();
-		testServer.produceIntoKafka(stream, topic,
-				new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, env.getConfig())),
-				props,
-				new KafkaPartitioner<Integer>() {
-					@Override
-					public int partition(Integer next, byte[] serializedKey, byte[] serializedValue, int numPartitions) {
-						return next % numPartitions;
-					}
-				});
-
-		env.execute("Scrambles int sequence generator");
-	}
-
-	// ------------------------------------------------------------------------
-
-	public static class InfiniteStringsGenerator extends Thread {
-
-		private final KafkaTestEnvironment server;
-
-		private final String topic;
-
-		private volatile Throwable error;
-
-		private volatile boolean running = true;
-
-
-		public InfiniteStringsGenerator(KafkaTestEnvironment server, String topic) {
-			this.server = server;
-			this.topic = topic;
-		}
-
-		@Override
-		public void run() {
-			// we manually feed data into the Kafka sink
-			RichFunction producer = null;
-			try {
-				Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(server.getBrokerConnectionString());
-				producerProperties.setProperty("retries", "3");
-				StreamTransformation<String> mockTransform = new MockStreamTransformation();
-				DataStream<String> stream = new DataStream<>(new DummyStreamExecutionEnvironment(), mockTransform);
-
-				StreamSink<String> sink = server.getProducerSink(
-						topic,
-						new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()),
-						producerProperties,
-						new FixedPartitioner<String>());
-
-				OneInputStreamOperatorTestHarness<String, Object> testHarness =
-						new OneInputStreamOperatorTestHarness<>(sink);
-
-				testHarness.open();
-
-				final StringBuilder bld = new StringBuilder();
-				final Random rnd = new Random();
-
-				while (running) {
-					bld.setLength(0);
-
-					int len = rnd.nextInt(100) + 1;
-					for (int i = 0; i < len; i++) {
-						bld.append((char) (rnd.nextInt(20) + 'a') );
-					}
-
-					String next = bld.toString();
-					testHarness.processElement(new StreamRecord<>(next));
-				}
-			}
-			catch (Throwable t) {
-				this.error = t;
-			}
-			finally {
-				if (producer != null) {
-					try {
-						producer.close();
-					}
-					catch (Throwable t) {
-						// ignore
-					}
-				}
-			}
-		}
-
-		public void shutdown() {
-			this.running = false;
-			this.interrupt();
-		}
-
-		public Throwable getError() {
-			return this.error;
-		}
-
-		private static class MockStreamTransformation extends StreamTransformation<String> {
-			public MockStreamTransformation() {
-				super("MockTransform", TypeInfoParser.<String>parse("String"), 1);
-			}
-
-			@Override
-			public void setChainingStrategy(ChainingStrategy strategy) {
-
-			}
-
-			@Override
-			public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-				return null;
-			}
-		}
-
-		public static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment {
-
-			@Override
-			public JobExecutionResult execute(String jobName) throws Exception {
-				return null;
-			}
-		}
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java
deleted file mode 100644
index 055326d..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-
-import java.util.Properties;
-
-public class FakeStandardProducerConfig {
-
-	public static Properties get() {
-		Properties p = new Properties();
-		p.setProperty("bootstrap.servers", "localhost:12345");
-		p.setProperty("key.serializer", ByteArraySerializer.class.getName());
-		p.setProperty("value.serializer", ByteArraySerializer.class.getName());
-		return p;
-	}
-
-}

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

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

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

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

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

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

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


[21/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
new file mode 100644
index 0000000..4d1acb4
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.hadoopcompatibility.mapred;
+
+import java.io.IOException;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.hadoopcompatibility.mapred.HadoopMapFunction;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class HadoopMapFunctionITCase extends MultipleProgramsTestBase {
+
+	public HadoopMapFunctionITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Test
+	public void testNonPassingMapper() throws Exception{
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env);
+		DataSet<Tuple2<IntWritable, Text>> nonPassingFlatMapDs = ds.
+				flatMap(new HadoopMapFunction<IntWritable, Text, IntWritable, Text>(new NonPassingMapper()));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		nonPassingFlatMapDs.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE);
+		env.execute();
+
+		compareResultsByLinesInMemory("\n", resultPath);
+	}
+
+	@Test
+	public void testDataDuplicatingMapper() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env);
+		DataSet<Tuple2<IntWritable, Text>> duplicatingFlatMapDs = ds.
+				flatMap(new HadoopMapFunction<IntWritable, Text, IntWritable, Text>(new DuplicatingMapper()));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		duplicatingFlatMapDs.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE);
+		env.execute();
+
+		String expected = "(1,Hi)\n" + "(1,HI)\n" +
+				"(2,Hello)\n" + "(2,HELLO)\n" +
+				"(3,Hello world)\n" + "(3,HELLO WORLD)\n" +
+				"(4,Hello world, how are you?)\n" + "(4,HELLO WORLD, HOW ARE YOU?)\n" +
+				"(5,I am fine.)\n" + "(5,I AM FINE.)\n" +
+				"(6,Luke Skywalker)\n" + "(6,LUKE SKYWALKER)\n" +
+				"(7,Comment#1)\n" + "(7,COMMENT#1)\n" +
+				"(8,Comment#2)\n" + "(8,COMMENT#2)\n" +
+				"(9,Comment#3)\n" + "(9,COMMENT#3)\n" +
+				"(10,Comment#4)\n" + "(10,COMMENT#4)\n" +
+				"(11,Comment#5)\n" + "(11,COMMENT#5)\n" +
+				"(12,Comment#6)\n" + "(12,COMMENT#6)\n" +
+				"(13,Comment#7)\n" + "(13,COMMENT#7)\n" +
+				"(14,Comment#8)\n" + "(14,COMMENT#8)\n" +
+				"(15,Comment#9)\n" + "(15,COMMENT#9)\n" +
+				"(16,Comment#10)\n" + "(16,COMMENT#10)\n" +
+				"(17,Comment#11)\n" + "(17,COMMENT#11)\n" +
+				"(18,Comment#12)\n" + "(18,COMMENT#12)\n" +
+				"(19,Comment#13)\n" + "(19,COMMENT#13)\n" +
+				"(20,Comment#14)\n" + "(20,COMMENT#14)\n" +
+				"(21,Comment#15)\n" + "(21,COMMENT#15)\n";
+
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+
+	@Test
+	public void testConfigurableMapper() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		JobConf conf = new JobConf();
+		conf.set("my.filterPrefix", "Hello");
+
+		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env);
+		DataSet<Tuple2<IntWritable, Text>> hellos = ds.
+				flatMap(new HadoopMapFunction<IntWritable, Text, IntWritable, Text>(new ConfigurableMapper(), conf));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		hellos.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE);
+		env.execute();
+
+		String expected = "(2,Hello)\n" +
+				"(3,Hello world)\n" +
+				"(4,Hello world, how are you?)\n";
+
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+	
+
+	
+	public static class NonPassingMapper implements Mapper<IntWritable, Text, IntWritable, Text> {
+		
+		@Override
+		public void map(final IntWritable k, final Text v, 
+				final OutputCollector<IntWritable, Text> out, final Reporter r) throws IOException {
+			if ( v.toString().contains("bananas") ) {
+				out.collect(k,v);
+			}
+		}
+		
+		@Override
+		public void configure(final JobConf arg0) { }
+
+		@Override
+		public void close() throws IOException { }
+	}
+	
+	public static class DuplicatingMapper implements Mapper<IntWritable, Text, IntWritable, Text> {
+		
+		@Override
+		public void map(final IntWritable k, final Text v, 
+				final OutputCollector<IntWritable, Text> out, final Reporter r) throws IOException {
+			out.collect(k, v);
+			out.collect(k, new Text(v.toString().toUpperCase()));
+		}
+		
+		@Override
+		public void configure(final JobConf arg0) { }
+
+		@Override
+		public void close() throws IOException { }
+	}
+	
+	public static class ConfigurableMapper implements Mapper<IntWritable, Text, IntWritable, Text> {
+		private String filterPrefix;
+		
+		@Override
+		public void map(IntWritable k, Text v, OutputCollector<IntWritable, Text> out, Reporter r)
+				throws IOException {
+			if(v.toString().startsWith(filterPrefix)) {
+				out.collect(k, v);
+			}
+		}
+		
+		@Override
+		public void configure(JobConf c) {
+			filterPrefix = c.get("my.filterPrefix");
+		}
+
+		@Override
+		public void close() throws IOException { }
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
new file mode 100644
index 0000000..13d971c
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.hadoopcompatibility.mapred;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceCombineFunction;
+import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceFunction;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.hamcrest.core.IsEqual;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class HadoopReduceCombineFunctionITCase extends MultipleProgramsTestBase {
+
+	public HadoopReduceCombineFunctionITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Test
+	public void testStandardCountingWithCombiner() throws Exception{
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<IntWritable, IntWritable>> ds = HadoopTestData.getKVPairDataSet(env).
+				map(new Mapper1());
+
+		DataSet<Tuple2<IntWritable, IntWritable>> counts = ds.
+				groupBy(0).
+				reduceGroup(new HadoopReduceCombineFunction<IntWritable, IntWritable, IntWritable, IntWritable>(
+						new SumReducer(), new SumReducer()));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		counts.writeAsText(resultPath);
+		env.execute();
+
+		String expected = "(0,5)\n"+
+				"(1,6)\n" +
+				"(2,6)\n" +
+				"(3,4)\n";
+
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+
+	@Test
+	public void testUngroupedHadoopReducer() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<IntWritable, IntWritable>> ds = HadoopTestData.getKVPairDataSet(env).
+				map(new Mapper2());
+
+		DataSet<Tuple2<IntWritable, IntWritable>> sum = ds.
+				reduceGroup(new HadoopReduceCombineFunction<IntWritable, IntWritable, IntWritable, IntWritable>(
+						new SumReducer(), new SumReducer()));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		sum.writeAsText(resultPath);
+		env.execute();
+
+		String expected = "(0,231)\n";
+
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+
+	@Test
+	public void testCombiner() throws Exception {
+		org.junit.Assume.assumeThat(mode, new IsEqual<TestExecutionMode>(TestExecutionMode.CLUSTER));
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<IntWritable, IntWritable>> ds = HadoopTestData.getKVPairDataSet(env).
+				map(new Mapper3());
+
+		DataSet<Tuple2<IntWritable, IntWritable>> counts = ds.
+				groupBy(0).
+				reduceGroup(new HadoopReduceCombineFunction<IntWritable, IntWritable, IntWritable, IntWritable>(
+						new SumReducer(), new KeyChangingReducer()));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		counts.writeAsText(resultPath);
+		env.execute();
+
+		String expected = "(0,5)\n"+
+				"(1,6)\n" +
+				"(2,5)\n" +
+				"(3,5)\n";
+
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+
+	@Test
+	public void testConfigurationViaJobConf() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		JobConf conf = new JobConf();
+		conf.set("my.cntPrefix", "Hello");
+
+		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env).
+				map(new Mapper4());
+
+		DataSet<Tuple2<IntWritable, IntWritable>> hellos = ds.
+				groupBy(0).
+				reduceGroup(new HadoopReduceFunction<IntWritable, Text, IntWritable, IntWritable>(
+						new ConfigurableCntReducer(), conf));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		hellos.writeAsText(resultPath);
+		env.execute();
+
+		// return expected result
+		String expected = "(0,0)\n"+
+				"(1,0)\n" +
+				"(2,1)\n" +
+				"(3,1)\n" +
+				"(4,1)\n";
+
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+	
+	public static class SumReducer implements Reducer<IntWritable, IntWritable, IntWritable, IntWritable> {
+
+		@Override
+		public void reduce(IntWritable k, Iterator<IntWritable> v, OutputCollector<IntWritable, IntWritable> out, Reporter r)
+				throws IOException {
+			
+			int sum = 0;
+			while(v.hasNext()) {
+				sum += v.next().get();
+			}
+			out.collect(k, new IntWritable(sum));
+		}
+		
+		@Override
+		public void configure(JobConf arg0) { }
+
+		@Override
+		public void close() throws IOException { }
+	}
+	
+	public static class KeyChangingReducer implements Reducer<IntWritable, IntWritable, IntWritable, IntWritable> {
+
+		@Override
+		public void reduce(IntWritable k, Iterator<IntWritable> v, OutputCollector<IntWritable, IntWritable> out, Reporter r)
+				throws IOException {
+			while(v.hasNext()) {
+				out.collect(new IntWritable(k.get() % 4), v.next());
+			}
+		}
+		
+		@Override
+		public void configure(JobConf arg0) { }
+
+		@Override
+		public void close() throws IOException { }
+	}
+	
+	public static class ConfigurableCntReducer implements Reducer<IntWritable, Text, IntWritable, IntWritable> {
+		private String countPrefix;
+		
+		@Override
+		public void reduce(IntWritable k, Iterator<Text> vs, OutputCollector<IntWritable, IntWritable> out, Reporter r)
+				throws IOException {
+			int commentCnt = 0;
+			while(vs.hasNext()) {
+				String v = vs.next().toString();
+				if(v.startsWith(this.countPrefix)) {
+					commentCnt++;
+				}
+			}
+			out.collect(k, new IntWritable(commentCnt));
+		}
+		
+		@Override
+		public void configure(final JobConf c) { 
+			this.countPrefix = c.get("my.cntPrefix");
+		}
+
+		@Override
+		public void close() throws IOException { }
+	}
+
+	public static class Mapper1 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable,
+			IntWritable>> {
+		private static final long serialVersionUID = 1L;
+		Tuple2<IntWritable,IntWritable> outT = new Tuple2<IntWritable,IntWritable>();
+		@Override
+		public Tuple2<IntWritable, IntWritable> map(Tuple2<IntWritable, Text> v)
+		throws Exception {
+			outT.f0 = new IntWritable(v.f0.get() / 6);
+			outT.f1 = new IntWritable(1);
+			return outT;
+		}
+	}
+
+	public static class Mapper2 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable,
+			IntWritable>> {
+		private static final long serialVersionUID = 1L;
+		Tuple2<IntWritable,IntWritable> outT = new Tuple2<IntWritable,IntWritable>();
+		@Override
+		public Tuple2<IntWritable, IntWritable> map(Tuple2<IntWritable, Text> v)
+		throws Exception {
+			outT.f0 = new IntWritable(0);
+			outT.f1 = v.f0;
+			return outT;
+		}
+	}
+
+	public static class Mapper3 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable, IntWritable>> {
+		private static final long serialVersionUID = 1L;
+		Tuple2<IntWritable,IntWritable> outT = new Tuple2<IntWritable,IntWritable>();
+		@Override
+		public Tuple2<IntWritable, IntWritable> map(Tuple2<IntWritable, Text> v)
+		throws Exception {
+			outT.f0 = v.f0;
+			outT.f1 = new IntWritable(1);
+			return outT;
+		}
+	}
+
+	public static class Mapper4 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable, Text>> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Tuple2<IntWritable, Text> map(Tuple2<IntWritable, Text> v)
+		throws Exception {
+			v.f0 = new IntWritable(v.f0.get() % 5);
+			return v;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
new file mode 100644
index 0000000..abc0e9c
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.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.
+ */
+
+package org.apache.flink.test.hadoopcompatibility.mapred;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceFunction;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class HadoopReduceFunctionITCase extends MultipleProgramsTestBase {
+
+	public HadoopReduceFunctionITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Test
+	public void testStandardGrouping() throws Exception{
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env).
+				map(new Mapper1());
+
+		DataSet<Tuple2<IntWritable, IntWritable>> commentCnts = ds.
+				groupBy(0).
+				reduceGroup(new HadoopReduceFunction<IntWritable, Text, IntWritable, IntWritable>(new CommentCntReducer()));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		commentCnts.writeAsText(resultPath);
+		env.execute();
+
+		String expected = "(0,0)\n"+
+				"(1,3)\n" +
+				"(2,5)\n" +
+				"(3,5)\n" +
+				"(4,2)\n";
+
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+
+	@Test
+	public void testUngroupedHadoopReducer() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env);
+
+		DataSet<Tuple2<IntWritable, IntWritable>> commentCnts = ds.
+				reduceGroup(new HadoopReduceFunction<IntWritable, Text, IntWritable, IntWritable>(new AllCommentCntReducer()));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		commentCnts.writeAsText(resultPath);
+		env.execute();
+
+		String expected = "(42,15)\n";
+
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+
+	@Test
+	public void testConfigurationViaJobConf() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		JobConf conf = new JobConf();
+		conf.set("my.cntPrefix", "Hello");
+
+		DataSet<Tuple2<IntWritable, Text>> ds = HadoopTestData.getKVPairDataSet(env).
+				map(new Mapper2());
+
+		DataSet<Tuple2<IntWritable, IntWritable>> helloCnts = ds.
+				groupBy(0).
+				reduceGroup(new HadoopReduceFunction<IntWritable, Text, IntWritable, IntWritable>(
+						new ConfigurableCntReducer(), conf));
+
+		String resultPath = tempFolder.newFile().toURI().toString();
+
+		helloCnts.writeAsText(resultPath);
+		env.execute();
+
+		String expected = "(0,0)\n"+
+				"(1,0)\n" +
+				"(2,1)\n" +
+				"(3,1)\n" +
+				"(4,1)\n";
+
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+	
+	public static class CommentCntReducer implements Reducer<IntWritable, Text, IntWritable, IntWritable> {
+		
+		@Override
+		public void reduce(IntWritable k, Iterator<Text> vs, OutputCollector<IntWritable, IntWritable> out, Reporter r)
+				throws IOException {
+			int commentCnt = 0;
+			while(vs.hasNext()) {
+				String v = vs.next().toString();
+				if(v.startsWith("Comment")) {
+					commentCnt++;
+				}
+			}
+			out.collect(k, new IntWritable(commentCnt));
+		}
+		
+		@Override
+		public void configure(final JobConf arg0) { }
+
+		@Override
+		public void close() throws IOException { }
+	}
+	
+	public static class AllCommentCntReducer implements Reducer<IntWritable, Text, IntWritable, IntWritable> {
+		
+		@Override
+		public void reduce(IntWritable k, Iterator<Text> vs, OutputCollector<IntWritable, IntWritable> out, Reporter r)
+				throws IOException {
+			int commentCnt = 0;
+			while(vs.hasNext()) {
+				String v = vs.next().toString();
+				if(v.startsWith("Comment")) {
+					commentCnt++;
+				}
+			}
+			out.collect(new IntWritable(42), new IntWritable(commentCnt));
+		}
+		
+		@Override
+		public void configure(final JobConf arg0) { }
+
+		@Override
+		public void close() throws IOException { }
+	}
+	
+	public static class ConfigurableCntReducer implements Reducer<IntWritable, Text, IntWritable, IntWritable> {
+		private String countPrefix;
+		
+		@Override
+		public void reduce(IntWritable k, Iterator<Text> vs, OutputCollector<IntWritable, IntWritable> out, Reporter r)
+				throws IOException {
+			int commentCnt = 0;
+			while(vs.hasNext()) {
+				String v = vs.next().toString();
+				if(v.startsWith(this.countPrefix)) {
+					commentCnt++;
+				}
+			}
+			out.collect(k, new IntWritable(commentCnt));
+		}
+		
+		@Override
+		public void configure(final JobConf c) { 
+			this.countPrefix = c.get("my.cntPrefix");
+		}
+
+		@Override
+		public void close() throws IOException { }
+	}
+
+	public static class Mapper1 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable, Text>> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Tuple2<IntWritable, Text> map(Tuple2<IntWritable, Text> v)
+		throws Exception {
+			v.f0 = new IntWritable(v.f0.get() / 5);
+			return v;
+		}
+	}
+
+	public static class Mapper2 implements MapFunction<Tuple2<IntWritable, Text>, Tuple2<IntWritable, Text>> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Tuple2<IntWritable, Text> map(Tuple2<IntWritable, Text> v)
+		throws Exception {
+			v.f0 = new IntWritable(v.f0.get() % 5);
+			return v;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java
new file mode 100644
index 0000000..eed6f8f
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.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.test.hadoopcompatibility.mapred;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+
+public class HadoopTestData {
+
+	public static DataSet<Tuple2<IntWritable, Text>> getKVPairDataSet(ExecutionEnvironment env) {
+		
+		List<Tuple2<IntWritable, Text>> data = new ArrayList<Tuple2<IntWritable, Text>>();
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(1),new Text("Hi")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(2),new Text("Hello")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(3),new Text("Hello world")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(4),new Text("Hello world, how are you?")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(5),new Text("I am fine.")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(6),new Text("Luke Skywalker")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(7),new Text("Comment#1")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(8),new Text("Comment#2")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(9),new Text("Comment#3")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(10),new Text("Comment#4")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(11),new Text("Comment#5")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(12),new Text("Comment#6")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(13),new Text("Comment#7")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(14),new Text("Comment#8")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(15),new Text("Comment#9")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(16),new Text("Comment#10")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(17),new Text("Comment#11")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(18),new Text("Comment#12")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(19),new Text("Comment#13")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(20),new Text("Comment#14")));
+		data.add(new Tuple2<IntWritable, Text>(new IntWritable(21),new Text("Comment#15")));
+		
+		Collections.shuffle(data);
+		
+		return env.fromCollection(data);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java
new file mode 100644
index 0000000..ce0143a
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.hadoopcompatibility.mapred.example;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat;
+import org.apache.flink.hadoopcompatibility.mapred.HadoopMapFunction;
+import org.apache.flink.api.java.hadoop.mapred.HadoopOutputFormat;
+import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceCombineFunction;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+
+
+
+/**
+ * Implements a word count which takes the input file and counts the number of
+ * occurrences of each word in the file and writes the result back to disk.
+ * 
+ * This example shows how to use Hadoop Input Formats, how to convert Hadoop Writables to 
+ * common Java types for better usage in a Flink job and how to use Hadoop Output Formats.
+ */
+public class HadoopMapredCompatWordCount {
+	
+	public static void main(String[] args) throws Exception {
+		if (args.length < 2) {
+			System.err.println("Usage: WordCount <input path> <result path>");
+			return;
+		}
+		
+		final String inputPath = args[0];
+		final String outputPath = args[1];
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		// Set up the Hadoop Input Format
+		HadoopInputFormat<LongWritable, Text> hadoopInputFormat = new HadoopInputFormat<LongWritable, Text>(new TextInputFormat(), LongWritable.class, Text.class, new JobConf());
+		TextInputFormat.addInputPath(hadoopInputFormat.getJobConf(), new Path(inputPath));
+		
+		// Create a Flink job with it
+		DataSet<Tuple2<LongWritable, Text>> text = env.createInput(hadoopInputFormat);
+		
+		DataSet<Tuple2<Text, LongWritable>> words = 
+				text.flatMap(new HadoopMapFunction<LongWritable, Text, Text, LongWritable>(new Tokenizer()))
+					.groupBy(0).reduceGroup(new HadoopReduceCombineFunction<Text, LongWritable, Text, LongWritable>(new Counter(), new Counter()));
+		
+		// Set up Hadoop Output Format
+		HadoopOutputFormat<Text, LongWritable> hadoopOutputFormat = 
+				new HadoopOutputFormat<Text, LongWritable>(new TextOutputFormat<Text, LongWritable>(), new JobConf());
+		hadoopOutputFormat.getJobConf().set("mapred.textoutputformat.separator", " ");
+		TextOutputFormat.setOutputPath(hadoopOutputFormat.getJobConf(), new Path(outputPath));
+		
+		// Output & Execute
+		words.output(hadoopOutputFormat).setParallelism(1);
+		env.execute("Hadoop Compat WordCount");
+	}
+	
+	
+	public static final class Tokenizer implements Mapper<LongWritable, Text, Text, LongWritable> {
+
+		@Override
+		public void map(LongWritable k, Text v, OutputCollector<Text, LongWritable> out, Reporter rep) 
+				throws IOException {
+			// normalize and split the line
+			String line = v.toString();
+			String[] tokens = line.toLowerCase().split("\\W+");
+			
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new Text(token), new LongWritable(1l));
+				}
+			}
+		}
+		
+		@Override
+		public void configure(JobConf arg0) { }
+		
+		@Override
+		public void close() throws IOException { }
+		
+	}
+	
+	public static final class Counter implements Reducer<Text, LongWritable, Text, LongWritable> {
+
+		@Override
+		public void reduce(Text k, Iterator<LongWritable> vs, OutputCollector<Text, LongWritable> out, Reporter rep)
+				throws IOException {
+			
+			long cnt = 0;
+			while(vs.hasNext()) {
+				cnt += vs.next().get();
+			}
+			out.collect(k, new LongWritable(cnt));
+			
+		}
+		
+		@Override
+		public void configure(JobConf arg0) { }
+		
+		@Override
+		public void close() throws IOException { }
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java
new file mode 100644
index 0000000..524318c
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.hadoopcompatibility.mapred.wrapper;
+
+import java.util.ArrayList;
+import java.util.NoSuchElementException;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.WritableSerializer;
+import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator;
+import org.apache.hadoop.io.IntWritable;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class HadoopTupleUnwrappingIteratorTest {
+
+	@Test
+	public void testValueIterator() {
+		
+		HadoopTupleUnwrappingIterator<IntWritable, IntWritable> valIt = 
+				new HadoopTupleUnwrappingIterator<IntWritable, IntWritable>(new WritableSerializer
+						<IntWritable>(IntWritable.class));
+		
+		// many values
+		
+		ArrayList<Tuple2<IntWritable, IntWritable>> tList = new ArrayList<Tuple2<IntWritable, IntWritable>>();
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(1)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(2)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(3)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(4)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(5)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(6)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(7)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(1),new IntWritable(8)));
+		
+		int expectedKey = 1;
+		int[] expectedValues = new int[] {1,2,3,4,5,6,7,8};
+		
+		valIt.set(tList.iterator());
+		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		for(int expectedValue : expectedValues) {
+			Assert.assertTrue(valIt.hasNext());
+			Assert.assertTrue(valIt.hasNext());
+			Assert.assertTrue(valIt.next().get() == expectedValue);
+			Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		}
+		Assert.assertFalse(valIt.hasNext());
+		Assert.assertFalse(valIt.hasNext());
+		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		
+		// one value
+		
+		tList.clear();
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(2),new IntWritable(10)));
+		
+		expectedKey = 2;
+		expectedValues = new int[]{10};
+		
+		valIt.set(tList.iterator());
+		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		for(int expectedValue : expectedValues) {
+			Assert.assertTrue(valIt.hasNext());
+			Assert.assertTrue(valIt.hasNext());
+			Assert.assertTrue(valIt.next().get() == expectedValue);
+			Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		}
+		Assert.assertFalse(valIt.hasNext());
+		Assert.assertFalse(valIt.hasNext());
+		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		
+		// more values
+		
+		tList.clear();
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(3),new IntWritable(10)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(3),new IntWritable(4)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(3),new IntWritable(7)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(3),new IntWritable(9)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(21)));
+		
+		expectedKey = 3;
+		expectedValues = new int[]{10,4,7,9,21};
+		
+		valIt.set(tList.iterator());
+		Assert.assertTrue(valIt.hasNext());
+		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		for(int expectedValue : expectedValues) {
+			Assert.assertTrue(valIt.hasNext());
+			Assert.assertTrue(valIt.hasNext());
+			Assert.assertTrue(valIt.next().get() == expectedValue);
+			Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		}
+		Assert.assertFalse(valIt.hasNext());
+		Assert.assertFalse(valIt.hasNext());
+		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		
+		// no has next calls
+		
+		tList.clear();
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(5)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(8)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(42)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(-1)));
+		tList.add(new Tuple2<IntWritable, IntWritable>(new IntWritable(4),new IntWritable(0)));
+		
+		expectedKey = 4;
+		expectedValues = new int[]{5,8,42,-1,0};
+		
+		valIt.set(tList.iterator());
+		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+		for(int expectedValue : expectedValues) {
+			Assert.assertTrue(valIt.next().get() == expectedValue);
+		}
+		try {
+			valIt.next();
+			Assert.fail();
+		} catch (NoSuchElementException nsee) {
+			// expected
+		}
+		Assert.assertFalse(valIt.hasNext());
+		Assert.assertTrue(valIt.getCurrentKey().get() == expectedKey);
+	}
+	
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java
new file mode 100644
index 0000000..ed83d78
--- /dev/null
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.hadoopcompatibility.mapreduce.example;
+
+import org.apache.flink.api.java.aggregation.Aggregations;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat;
+import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat;
+
+/**
+ * Implements a word count which takes the input file and counts the number of
+ * occurrences of each word in the file and writes the result back to disk.
+ * 
+ * This example shows how to use Hadoop Input Formats, how to convert Hadoop Writables to 
+ * common Java types for better usage in a Flink job and how to use Hadoop Output Formats.
+ */
+@SuppressWarnings("serial")
+public class WordCount {
+	
+	public static void main(String[] args) throws Exception {
+		if (args.length < 2) {
+			System.err.println("Usage: WordCount <input path> <result path>");
+			return;
+		}
+		
+		final String inputPath = args[0];
+		final String outputPath = args[1];
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		// Set up the Hadoop Input Format
+		Job job = Job.getInstance();
+		HadoopInputFormat<LongWritable, Text> hadoopInputFormat = new HadoopInputFormat<LongWritable, Text>(new TextInputFormat(), LongWritable.class, Text.class, job);
+		TextInputFormat.addInputPath(job, new Path(inputPath));
+		
+		// Create a Flink job with it
+		DataSet<Tuple2<LongWritable, Text>> text = env.createInput(hadoopInputFormat);
+		
+		// Tokenize the line and convert from Writable "Text" to String for better handling
+		DataSet<Tuple2<String, Integer>> words = text.flatMap(new Tokenizer());
+		
+		// Sum up the words
+		DataSet<Tuple2<String, Integer>> result = words.groupBy(0).aggregate(Aggregations.SUM, 1);
+		
+		// Convert String back to Writable "Text" for use with Hadoop Output Format
+		DataSet<Tuple2<Text, IntWritable>> hadoopResult = result.map(new HadoopDatatypeMapper());
+		
+		// Set up Hadoop Output Format
+		HadoopOutputFormat<Text, IntWritable> hadoopOutputFormat = new HadoopOutputFormat<Text, IntWritable>(new TextOutputFormat<Text, IntWritable>(), job);
+		hadoopOutputFormat.getConfiguration().set("mapreduce.output.textoutputformat.separator", " ");
+		hadoopOutputFormat.getConfiguration().set("mapred.textoutputformat.separator", " "); // set the value for both, since this test
+		TextOutputFormat.setOutputPath(job, new Path(outputPath));
+		
+		// Output & Execute
+		hadoopResult.output(hadoopOutputFormat);
+		env.execute("Word Count");
+	}
+	
+	/**
+	 * Splits a line into words and converts Hadoop Writables into normal Java data types.
+	 */
+	public static final class Tokenizer extends RichFlatMapFunction<Tuple2<LongWritable, Text>, Tuple2<String, Integer>> {
+		
+		@Override
+		public void flatMap(Tuple2<LongWritable, Text> value, Collector<Tuple2<String, Integer>> out) {
+			// normalize and split the line
+			String line = value.f1.toString();
+			String[] tokens = line.toLowerCase().split("\\W+");
+			
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new Tuple2<String, Integer>(token, 1));
+				}
+			}
+		}
+	}
+	
+	/**
+	 * Converts Java data types to Hadoop Writables.
+	 */
+	public static final class HadoopDatatypeMapper extends RichMapFunction<Tuple2<String, Integer>, Tuple2<Text, IntWritable>> {
+		
+		@Override
+		public Tuple2<Text, IntWritable> map(Tuple2<String, Integer> value) throws Exception {
+			return new Tuple2<Text, IntWritable>(new Text(value.f0), new IntWritable(value.f1));
+		}
+		
+	}
+	
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/pom.xml b/flink-connectors/flink-hbase/pom.xml
new file mode 100644
index 0000000..9b1e174
--- /dev/null
+++ b/flink-connectors/flink-hbase/pom.xml
@@ -0,0 +1,264 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-hbase_2.10</artifactId>
+	<name>flink-hbase</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<hbase.version>1.2.3</hbase.version>
+	</properties>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<version>2.19.1</version>
+				<configuration>
+					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
+					<forkCount>1</forkCount>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<!-- Disable inherited shade-flink because of a problem in the shade plugin -->
+						<!-- When enabled you'll run into an infinite loop creating the dependency-reduced-pom.xml -->
+						<!-- Seems similar to https://issues.apache.org/jira/browse/MSHADE-148 -->
+						<id>shade-flink</id>
+						<phase>none</phase>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+
+	<dependencies>
+
+		<!-- core dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-hadoop2</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+
+			<!--Exclude Guava in order to run the HBaseMiniCluster during testing-->
+			<exclusions>
+				<exclusion>
+					<groupId>com.google.guava</groupId>
+					<artifactId>guava</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<!-- HBase server needed for TableOutputFormat -->
+		<!-- TODO implement bulk output format for HBase -->
+		<dependency>
+			<groupId>org.apache.hbase</groupId>
+			<artifactId>hbase-server</artifactId>
+			<version>${hbase.version}</version>
+			<exclusions>
+				<!-- Remove unneeded dependency, which is conflicting with our jetty-util version. -->
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty-util</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty-sslengine</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jsp-2.1</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jsp-api-2.1</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>servlet-api-2.5</artifactId>
+				</exclusion>
+				<!-- The hadoop dependencies are handled through flink-shaded-hadoop -->
+				<exclusion>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-common</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-auth</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-annotations</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-mapreduce-client-core</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-client</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-hdfs</artifactId>
+				</exclusion>
+				<!-- Bug in hbase annotations, can be removed when fixed. See FLINK-2153. -->
+				<exclusion>
+					<groupId>org.apache.hbase</groupId>
+					<artifactId>hbase-annotations</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<exclusions>
+				<exclusion>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-core</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<exclusions>
+				<exclusion>
+					<groupId>org.apache.flink</groupId>
+					<artifactId>flink-shaded-include-yarn_2.10</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<!-- Test dependencies are only available for Hadoop-2. -->
+		<dependency>
+			<groupId>org.apache.hbase</groupId>
+			<artifactId>hbase-server</artifactId>
+			<version>${hbase.version}</version>
+			<classifier>tests</classifier>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-minicluster</artifactId>
+			<version>${hadoop.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hbase</groupId>
+			<artifactId>hbase-hadoop-compat</artifactId>
+			<version>${hbase.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<version>${hadoop.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hbase</groupId>
+			<artifactId>hbase-hadoop2-compat</artifactId>
+			<version>${hbase.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+	</dependencies>
+
+	<profiles>
+		<profile>
+			<id>cdh5.1.3</id>
+			<properties>
+				<hbase.version>0.98.1-cdh5.1.3</hbase.version>
+				<hadoop.version>2.3.0-cdh5.1.3</hadoop.version>
+				<!-- Cloudera use different versions for hadoop core and commons-->
+				<!-- This profile could be removed if Cloudera fix this mismatch! -->
+				<hadoop.core.version>2.3.0-mr1-cdh5.1.3</hadoop.core.version>
+			</properties>
+			<dependencyManagement>
+				<dependencies>
+					<dependency>
+						<groupId>org.apache.hadoop</groupId>
+						<artifactId>hadoop-core</artifactId>
+						<version>${hadoop.core.version}</version>
+					</dependency>
+				</dependencies>
+			</dependencyManagement>
+		</profile>
+
+	</profiles>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
new file mode 100644
index 0000000..35b0a7c
--- /dev/null
+++ b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.addons.hbase;
+
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
+import org.apache.flink.api.common.io.RichInputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * {@link InputFormat} subclass that wraps the access for HTables.
+ */
+public abstract class TableInputFormat<T extends Tuple> extends RichInputFormat<T, TableInputSplit> {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(TableInputFormat.class);
+
+	/** helper variable to decide whether the input is exhausted or not */
+	private boolean endReached = false;
+
+	protected transient HTable table = null;
+	protected transient Scan scan = null;
+
+	/** HBase iterator wrapper */
+	private ResultScanner resultScanner = null;
+
+	private byte[] lastRow;
+	private int scannedRows;
+
+	/**
+	 * Returns an instance of Scan that retrieves the required subset of records from the HBase table.
+	 * @return The appropriate instance of Scan for this usecase.
+	 */
+	protected abstract Scan getScanner();
+
+	/**
+	 * What table is to be read.
+	 * Per instance of a TableInputFormat derivative only a single tablename is possible.
+	 * @return The name of the table
+	 */
+	protected abstract String getTableName();
+
+	/**
+	 * The output from HBase is always an instance of {@link Result}.
+	 * This method is to copy the data in the Result instance into the required {@link Tuple}
+	 * @param r The Result instance from HBase that needs to be converted
+	 * @return The approriate instance of {@link Tuple} that contains the needed information.
+	 */
+	protected abstract T mapResultToTuple(Result r);
+
+	/**
+	 * Creates a {@link Scan} object and opens the {@link HTable} connection.
+	 * These are opened here because they are needed in the createInputSplits
+	 * which is called before the openInputFormat method.
+	 * So the connection is opened in {@link #configure(Configuration)} and closed in {@link #closeInputFormat()}.
+	 *
+	 * @param parameters The configuration that is to be used
+	 * @see Configuration
+	 */
+	@Override
+	public void configure(Configuration parameters) {
+		table = createTable();
+		if (table != null) {
+			scan = getScanner();
+		}
+	}
+
+	/**
+	 * Create an {@link HTable} instance and set it into this format
+	 */
+	private HTable createTable() {
+		LOG.info("Initializing HBaseConfiguration");
+		//use files found in the classpath
+		org.apache.hadoop.conf.Configuration hConf = HBaseConfiguration.create();
+
+		try {
+			return new HTable(hConf, getTableName());
+		} catch (Exception e) {
+			LOG.error("Error instantiating a new HTable instance", e);
+		}
+		return null;
+	}
+
+	@Override
+	public void open(TableInputSplit split) throws IOException {
+		if (table == null) {
+			throw new IOException("The HBase table has not been opened!");
+		}
+		if (scan == null) {
+			throw new IOException("getScanner returned null");
+		}
+		if (split == null) {
+			throw new IOException("Input split is null!");
+		}
+
+		logSplitInfo("opening", split);
+		scan.setStartRow(split.getStartRow());
+		lastRow = split.getEndRow();
+		scan.setStopRow(lastRow);
+
+		resultScanner = table.getScanner(scan);
+		endReached = false;
+		scannedRows = 0;
+	}
+
+	@Override
+	public boolean reachedEnd() throws IOException {
+		return endReached;
+	}
+
+	@Override
+	public T nextRecord(T reuse) throws IOException {
+		if (resultScanner == null) {
+			throw new IOException("No table result scanner provided!");
+		}
+		try {
+			Result res = resultScanner.next();
+			if (res != null) {
+				scannedRows++;
+				lastRow = res.getRow();
+				return mapResultToTuple(res);
+			}
+		} catch (Exception e) {
+			resultScanner.close();
+			//workaround for timeout on scan
+			LOG.warn("Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
+			scan.setStartRow(lastRow);
+			resultScanner = table.getScanner(scan);
+			Result res = resultScanner.next();
+			if (res != null) {
+				scannedRows++;
+				lastRow = res.getRow();
+				return mapResultToTuple(res);
+			}
+		}
+
+		endReached = true;
+		return null;
+	}
+
+	@Override
+	public void close() throws IOException {
+		LOG.info("Closing split (scanned {} rows)", scannedRows);
+		lastRow = null;
+		try {
+			if (resultScanner != null) {
+				resultScanner.close();
+			}
+		} finally {
+			resultScanner = null;
+		}
+	}
+
+	@Override
+	public void closeInputFormat() throws IOException {
+		try {
+			if (table != null) {
+				table.close();
+			}
+		} finally {
+			table = null;
+		}
+	}
+
+	@Override
+	public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
+		if (table == null) {
+			throw new IOException("The HBase table has not been opened!");
+		}
+		if (scan == null) {
+			throw new IOException("getScanner returned null");
+		}
+
+		//Gets the starting and ending row keys for every region in the currently open table
+		final Pair<byte[][], byte[][]> keys = table.getStartEndKeys();
+		if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
+			throw new IOException("Expecting at least one region.");
+		}
+		final byte[] startRow = scan.getStartRow();
+		final byte[] stopRow = scan.getStopRow();
+		final boolean scanWithNoLowerBound = startRow.length == 0;
+		final boolean scanWithNoUpperBound = stopRow.length == 0;
+
+		final List<TableInputSplit> splits = new ArrayList<TableInputSplit>(minNumSplits);
+		for (int i = 0; i < keys.getFirst().length; i++) {
+			final byte[] startKey = keys.getFirst()[i];
+			final byte[] endKey = keys.getSecond()[i];
+			final String regionLocation = table.getRegionLocation(startKey, false).getHostnamePort();
+			//Test if the given region is to be included in the InputSplit while splitting the regions of a table
+			if (!includeRegionInSplit(startKey, endKey)) {
+				continue;
+			}
+			//Finds the region on which the given row is being served
+			final String[] hosts = new String[]{regionLocation};
+
+			// determine if regions contains keys used by the scan
+			boolean isLastRegion = endKey.length == 0;
+			if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) &&
+				(scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
+
+				final byte[] splitStart = scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0 ? startKey : startRow;
+				final byte[] splitStop = (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
+					&& !isLastRegion ? endKey : stopRow;
+				int id = splits.size();
+				final TableInputSplit split = new TableInputSplit(id, hosts, table.getTableName(), splitStart, splitStop);
+				splits.add(split);
+			}
+		}
+		LOG.info("Created " + splits.size() + " splits");
+		for (TableInputSplit split : splits) {
+			logSplitInfo("created", split);
+		}
+		return splits.toArray(new TableInputSplit[0]);
+	}
+
+	private void logSplitInfo(String action, TableInputSplit split) {
+		int splitId = split.getSplitNumber();
+		String splitStart = Bytes.toString(split.getStartRow());
+		String splitEnd = Bytes.toString(split.getEndRow());
+		String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
+		String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
+		String[] hostnames = split.getHostnames();
+		LOG.info("{} split (this={})[{}|{}|{}|{}]", action, this, splitId, hostnames, splitStartKey, splitStopKey);
+	}
+
+	/**
+	 * Test if the given region is to be included in the InputSplit while splitting the regions of a table.
+	 * <p>
+	 * This optimization is effective when there is a specific reasoning to exclude an entire region from the M-R job,
+	 * (and hence, not contributing to the InputSplit), given the start and end keys of the same. <br>
+	 * Useful when we need to remember the last-processed top record and revisit the [last, current) interval for M-R
+	 * processing, continuously. In addition to reducing InputSplits, reduces the load on the region server as well, due
+	 * to the ordering of the keys. <br>
+	 * <br>
+	 * Note: It is possible that <code>endKey.length() == 0 </code> , for the last (recent) region. <br>
+	 * Override this method, if you want to bulk exclude regions altogether from M-R. By default, no region is excluded(
+	 * i.e. all regions are included).
+	 *
+	 * @param startKey Start key of the region
+	 * @param endKey   End key of the region
+	 * @return true, if this region needs to be included as part of the input (default).
+	 */
+	protected boolean includeRegionInSplit(final byte[] startKey, final byte[] endKey) {
+		return true;
+	}
+
+	@Override
+	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
+		return new LocatableInputSplitAssigner(inputSplits);
+	}
+
+	@Override
+	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
new file mode 100644
index 0000000..75f0b9b
--- /dev/null
+++ b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.addons.hbase;
+
+import org.apache.flink.core.io.LocatableInputSplit;
+
+/**
+ * This class implements a input splits for HBase. Each table input split corresponds to a key range (low, high). All
+ * references to row below refer to the key of the row.
+ */
+public class TableInputSplit extends LocatableInputSplit {
+
+	private static final long serialVersionUID = 1L;
+
+	/** The name of the table to retrieve data from */
+	private final byte[] tableName;
+
+	/** The start row of the split. */
+	private final byte[] startRow;
+
+	/** The end row of the split. */
+	private final byte[] endRow;
+
+	/**
+	 * Creates a new table input split
+	 * 
+	 * @param splitNumber
+	 *        the number of the input split
+	 * @param hostnames
+	 *        the names of the hosts storing the data the input split refers to
+	 * @param tableName
+	 *        the name of the table to retrieve data from
+	 * @param startRow
+	 *        the start row of the split
+	 * @param endRow
+	 *        the end row of the split
+	 */
+	TableInputSplit(final int splitNumber, final String[] hostnames, final byte[] tableName, final byte[] startRow,
+			final byte[] endRow) {
+		super(splitNumber, hostnames);
+
+		this.tableName = tableName;
+		this.startRow = startRow;
+		this.endRow = endRow;
+	}
+
+	/**
+	 * Returns the table name.
+	 * 
+	 * @return The table name.
+	 */
+	public byte[] getTableName() {
+		return this.tableName;
+	}
+
+	/**
+	 * Returns the start row.
+	 * 
+	 * @return The start row.
+	 */
+	public byte[] getStartRow() {
+		return this.startRow;
+	}
+
+	/**
+	 * Returns the end row.
+	 * 
+	 * @return The end row.
+	 */
+	public byte[] getEndRow() {
+		return this.endRow;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseTestingClusterAutostarter.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseTestingClusterAutostarter.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseTestingClusterAutostarter.java
new file mode 100644
index 0000000..3d9f672
--- /dev/null
+++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseTestingClusterAutostarter.java
@@ -0,0 +1,238 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.addons.hbase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.ScannerCallable;
+import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.log4j.Level;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * By using this class as the super class of a set of tests you will have a HBase testing
+ * cluster available that is very suitable for writing tests for scanning and filtering against.
+ * This is usable by any downstream application because the HBase cluster is 'injected' because
+ * a dynamically generated hbase-site.xml is added to the classpath.
+ * Because of this classpath manipulation it is not possible to start a second testing cluster in the same JVM.
+ * So if you have this you should either put all hbase related tests in a single class or force surefire to
+ * setup a new JVM for each testclass.
+ * See: http://maven.apache.org/surefire/maven-surefire-plugin/examples/fork-options-and-parallel-execution.html
+ */
+//
+// NOTE: The code in this file is based on code from the
+// Apache HBase project, licensed under the Apache License v 2.0
+//
+// https://github.com/apache/hbase/blob/master/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java
+//
+public class HBaseTestingClusterAutostarter implements Serializable {
+
+	private static final Log LOG = LogFactory.getLog(HBaseTestingClusterAutostarter.class);
+
+	private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+	private static HBaseAdmin admin = null;
+	private static List<TableName> createdTables = new ArrayList<>();
+
+	private static boolean alreadyRegisteredTestCluster = false;
+
+	protected static void createTable(TableName tableName, byte[] columnFamilyName, byte[][] splitKeys) {
+		LOG.info("HBase minicluster: Creating table " + tableName.getNameAsString());
+
+		assertNotNull("HBaseAdmin is not initialized successfully.", admin);
+		HTableDescriptor desc = new HTableDescriptor(tableName);
+		HColumnDescriptor colDef = new HColumnDescriptor(columnFamilyName);
+		desc.addFamily(colDef);
+
+		try {
+			admin.createTable(desc, splitKeys);
+			createdTables.add(tableName);
+			assertTrue("Fail to create the table", admin.tableExists(tableName));
+		} catch (IOException e) {
+			assertNull("Exception found while creating table", e);
+		}
+	}
+
+	protected static HTable openTable(TableName tableName) throws IOException {
+		HTable table = (HTable) admin.getConnection().getTable(tableName);
+		assertTrue("Fail to create the table", admin.tableExists(tableName));
+		return table;
+	}
+
+	private static void deleteTables() {
+		if (admin != null) {
+			for (TableName tableName : createdTables) {
+				try {
+					if (admin.tableExists(tableName)) {
+						admin.disableTable(tableName);
+						admin.deleteTable(tableName);
+					}
+				} catch (IOException e) {
+					assertNull("Exception found deleting the table", e);
+				}
+			}
+		}
+	}
+
+	private static void initialize(Configuration conf) {
+		conf = HBaseConfiguration.create(conf);
+		conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
+		try {
+			admin = TEST_UTIL.getHBaseAdmin();
+		} catch (MasterNotRunningException e) {
+			assertNull("Master is not running", e);
+		} catch (ZooKeeperConnectionException e) {
+			assertNull("Cannot connect to ZooKeeper", e);
+		} catch (IOException e) {
+			assertNull("IOException", e);
+		}
+	}
+
+	@BeforeClass
+	public static void setUp() throws Exception {
+		LOG.info("HBase minicluster: Starting");
+		((Log4JLogger) RpcServer.LOG).getLogger().setLevel(Level.ALL);
+		((Log4JLogger) AbstractRpcClient.LOG).getLogger().setLevel(Level.ALL);
+		((Log4JLogger) ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
+
+		TEST_UTIL.startMiniCluster(1);
+
+		// https://issues.apache.org/jira/browse/HBASE-11711
+		TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", -1);
+
+		// Make sure the zookeeper quorum value contains the right port number (varies per run).
+		TEST_UTIL.getConfiguration().set("hbase.zookeeper.quorum", "localhost:" + TEST_UTIL.getZkCluster().getClientPort());
+
+		initialize(TEST_UTIL.getConfiguration());
+		LOG.info("HBase minicluster: Running");
+	}
+
+	private static File hbaseSiteXmlDirectory;
+	private static File hbaseSiteXmlFile;
+
+	/**
+	 * This dynamically generates a hbase-site.xml file that is added to the classpath.
+	 * This way this HBaseMinicluster can be used by an unmodified application.
+	 * The downside is that this cannot be 'unloaded' so you can have only one per JVM.
+	 */
+	public static void registerHBaseMiniClusterInClasspath() {
+		if (alreadyRegisteredTestCluster) {
+			fail("You CANNOT register a second HBase Testing cluster in the classpath of the SAME JVM");
+		}
+		File baseDir = new File(System.getProperty("java.io.tmpdir", "/tmp/"));
+		hbaseSiteXmlDirectory = new File(baseDir, "unittest-hbase-minicluster-" + Math.abs(new Random().nextLong()) + "/");
+
+		if (!hbaseSiteXmlDirectory.mkdirs()) {
+			fail("Unable to create output directory " + hbaseSiteXmlDirectory + " for the HBase minicluster");
+		}
+
+		assertNotNull("The ZooKeeper for the HBase minicluster is missing", TEST_UTIL.getZkCluster());
+
+		createHBaseSiteXml(hbaseSiteXmlDirectory, TEST_UTIL.getConfiguration().get("hbase.zookeeper.quorum"));
+		addDirectoryToClassPath(hbaseSiteXmlDirectory);
+
+		// Avoid starting it again.
+		alreadyRegisteredTestCluster = true;
+	}
+
+	private static void createHBaseSiteXml(File hbaseSiteXmlDirectory, String zookeeperQuorum) {
+		hbaseSiteXmlFile = new File(hbaseSiteXmlDirectory, "hbase-site.xml");
+		// Create the hbase-site.xml file for this run.
+		try {
+			String hbaseSiteXml = "<?xml version=\"1.0\"?>\n" +
+				"<?xml-stylesheet type=\"text/xsl\" href=\"configuration.xsl\"?>\n" +
+				"<configuration>\n" +
+				"  <property>\n" +
+				"    <name>hbase.zookeeper.quorum</name>\n" +
+				"    <value>" + zookeeperQuorum + "</value>\n" +
+				"  </property>\n" +
+				"</configuration>";
+			OutputStream fos = new FileOutputStream(hbaseSiteXmlFile);
+			fos.write(hbaseSiteXml.getBytes(StandardCharsets.UTF_8));
+			fos.close();
+		} catch (IOException e) {
+			fail("Unable to create " + hbaseSiteXmlFile);
+		}
+	}
+
+	private static void addDirectoryToClassPath(File directory) {
+		try {
+			// Get the classloader actually used by HBaseConfiguration
+			ClassLoader classLoader = HBaseConfiguration.create().getClassLoader();
+			if (!(classLoader instanceof URLClassLoader)) {
+				fail("We should get a URLClassLoader");
+			}
+
+			// Make the addURL method accessible
+			Method method = URLClassLoader.class.getDeclaredMethod("addURL", URL.class);
+			method.setAccessible(true);
+
+			// Add the directory where we put the hbase-site.xml to the classpath
+			method.invoke(classLoader, directory.toURI().toURL());
+		} catch (MalformedURLException | NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+			fail("Unable to add " + directory + " to classpath because of this exception: " + e.getMessage());
+		}
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		LOG.info("HBase minicluster: Shutting down");
+		deleteTables();
+		hbaseSiteXmlFile.delete();
+		hbaseSiteXmlDirectory.delete();
+		TEST_UTIL.shutdownMiniCluster();
+		LOG.info("HBase minicluster: Down");
+	}
+
+}


[38/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml
new file mode 100644
index 0000000..26352bb
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml
@@ -0,0 +1,205 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-kafka-0.10_2.10</artifactId>
+	<name>flink-connector-kafka-0.10</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<kafka.version>0.10.0.1</kafka.version>
+	</properties>
+
+	<dependencies>
+
+		<!-- core dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kafka-0.9_2.10</artifactId>
+			<version>${project.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>org.apache.kafka</groupId>
+					<artifactId>kafka_${scala.binary.version}</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<!-- Add Kafka 0.10.x as a dependency -->
+
+		<dependency>
+			<groupId>org.apache.kafka</groupId>
+			<artifactId>kafka-clients</artifactId>
+			<version>${kafka.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project,
+			won't depend on flink-table. -->
+			<optional>true</optional>
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kafka-0.9_2.10</artifactId>
+			<version>${project.version}</version>
+			<exclusions>
+				<!-- exclude Kafka dependencies -->
+				<exclusion>
+					<groupId>org.apache.kafka</groupId>
+					<artifactId>kafka_${scala.binary.version}</artifactId>
+				</exclusion>
+			</exclusions>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<version>${project.version}</version>
+			<exclusions>
+				<!-- exclude Kafka dependencies -->
+				<exclusion>
+					<groupId>org.apache.kafka</groupId>
+					<artifactId>kafka_${scala.binary.version}</artifactId>
+				</exclusion>
+			</exclusions>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- include 0.10 server for tests  -->
+			<groupId>org.apache.kafka</groupId>
+			<artifactId>kafka_${scala.binary.version}</artifactId>
+			<version>${kafka.version}</version>
+			<scope>test</scope>
+		</dependency>
+		
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-metrics-jmx</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+						<configuration>
+							<includes>
+								<include>**/KafkaTestEnvironmentImpl*</include>
+							</includes>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-source-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>attach-test-sources</id>
+						<goals>
+							<goal>test-jar-no-fork</goal>
+						</goals>
+						<configuration>
+							<includes>
+								<include>**/KafkaTestEnvironmentImpl*</include>
+							</includes>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
+					<forkCount>1</forkCount>
+					<argLine>-Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+	
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
new file mode 100644
index 0000000..a9ce336
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.internal.Kafka010Fetcher;
+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.DeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
+import org.apache.flink.util.SerializedValue;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+
+/**
+ * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
+ * Apache Kafka 0.10.x. The consumer can run in multiple parallel instances, each of which will pull
+ * data from one or more Kafka partitions. 
+ * 
+ * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
+ * during a failure, and that the computation processes elements "exactly once". 
+ * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
+ *
+ * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
+ * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
+ * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
+ * has consumed a topic.</p>
+ *
+ * <p>Please refer to Kafka's documentation for the available configuration properties:
+ * http://kafka.apache.org/documentation.html#newconsumerconfigs</p>
+ *
+ * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
+ * is constructed. That means that the client that submits the program needs to be able to
+ * reach the Kafka brokers or ZooKeeper.</p>
+ */
+public class FlinkKafkaConsumer010<T> extends FlinkKafkaConsumer09<T> {
+
+	private static final long serialVersionUID = 2324564345203409112L;
+
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.10.x
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
+	public FlinkKafkaConsumer010(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
+		this(Collections.singletonList(topic), valueDeserializer, props);
+	}
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.10.x
+	 *
+	 * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value
+	 * pairs, offsets, and topic names from Kafka.
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param deserializer
+	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
+	public FlinkKafkaConsumer010(String topic, KeyedDeserializationSchema<T> deserializer, Properties props) {
+		this(Collections.singletonList(topic), deserializer, props);
+	}
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.10.x
+	 *
+	 * This constructor allows passing multiple topics to the consumer.
+	 *
+	 * @param topics
+	 *           The Kafka topics to read from.
+	 * @param deserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties that are used to configure both the fetcher and the offset handler.
+	 */
+	public FlinkKafkaConsumer010(List<String> topics, DeserializationSchema<T> deserializer, Properties props) {
+		this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props);
+	}
+
+	/**
+	 * Creates a new Kafka streaming source consumer for Kafka 0.10.x
+	 *
+	 * This constructor allows passing multiple topics and a key/value deserialization schema.
+	 *
+	 * @param topics
+	 *           The Kafka topics to read from.
+	 * @param deserializer
+	 *           The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param props
+	 *           The properties that are used to configure both the fetcher and the offset handler.
+	 */
+	public FlinkKafkaConsumer010(List<String> topics, KeyedDeserializationSchema<T> deserializer, Properties props) {
+		super(topics, deserializer, props);
+	}
+
+	@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 Kafka010Fetcher<>(
+				sourceContext,
+				thisSubtaskPartitions,
+				watermarksPeriodic,
+				watermarksPunctuated,
+				runtimeContext.getProcessingTimeService(),
+				runtimeContext.getExecutionConfig().getAutoWatermarkInterval(),
+				runtimeContext.getUserCodeClassLoader(),
+				runtimeContext.isCheckpointingEnabled(),
+				runtimeContext.getTaskNameWithSubtasks(),
+				runtimeContext.getMetricGroup(),
+				deserializer,
+				properties,
+				pollTimeout,
+				useMetrics);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
new file mode 100644
index 0000000..cc0194b
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.apache.kafka.clients.producer.ProducerRecord;
+
+import java.util.Properties;
+
+import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase.getPropertiesFromBrokerList;
+
+
+/**
+ * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.10.x
+ *
+ * Implementation note: This producer is a hybrid between a regular regular sink function (a)
+ * and a custom operator (b).
+ *
+ * For (a), the class implements the SinkFunction and RichFunction interfaces.
+ * For (b), it extends the StreamTask class.
+ *
+ * Details about approach (a):
+ *
+ *  Pre Kafka 0.10 producers only follow approach (a), allowing users to use the producer using the
+ *  DataStream.addSink() method.
+ *  Since the APIs exposed in that variant do not allow accessing the the timestamp attached to the record
+ *  the Kafka 0.10 producer has a second invocation option, approach (b).
+ *
+ * Details about approach (b):
+ *  Kafka 0.10 supports writing the timestamp attached to a record to Kafka. When adding the
+ *  FlinkKafkaProducer010 using the FlinkKafkaProducer010.writeToKafkaWithTimestamps() method, the Kafka producer
+ *  can access the internal record timestamp of the record and write it to Kafka.
+ *
+ * All methods and constructors in this class are marked with the approach they are needed for.
+ */
+public class FlinkKafkaProducer010<T> extends StreamSink<T> implements SinkFunction<T>, RichFunction {
+
+	/**
+	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
+	 */
+	private boolean writeTimestampToKafka = false;
+
+	// ---------------------- "Constructors" for timestamp writing ------------------
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+	 * the topic.
+	 *
+	 * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
+	 *
+	 * @param inStream The stream to write to Kafka
+	 * @param topicId ID of the Kafka topic.
+	 * @param serializationSchema User defined serialization schema supporting key/value messages
+	 * @param producerConfig Properties with the producer configuration.
+	 */
+	public static <T> FlinkKafkaProducer010Configuration<T> writeToKafkaWithTimestamps(DataStream<T> inStream,
+																					String topicId,
+																					KeyedSerializationSchema<T> serializationSchema,
+																					Properties producerConfig) {
+		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FixedPartitioner<T>());
+	}
+
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
+	 * the topic.
+	 *
+	 * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
+	 *
+	 * @param inStream The stream to write to Kafka
+	 * @param topicId ID of the Kafka topic.
+	 * @param serializationSchema User defined (keyless) serialization schema.
+	 * @param producerConfig Properties with the producer configuration.
+	 */
+	public static <T> FlinkKafkaProducer010Configuration<T> writeToKafkaWithTimestamps(DataStream<T> inStream,
+																					String topicId,
+																					SerializationSchema<T> serializationSchema,
+																					Properties producerConfig) {
+		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner<T>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+	 * the topic.
+	 *
+	 * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
+	 *
+	 *  @param inStream The stream to write to Kafka
+	 *  @param topicId The name of the target topic
+	 *  @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
+	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
+	 */
+	public static <T> FlinkKafkaProducer010Configuration<T> writeToKafkaWithTimestamps(DataStream<T> inStream,
+																					String topicId,
+																					KeyedSerializationSchema<T> serializationSchema,
+																					Properties producerConfig,
+																					KafkaPartitioner<T> customPartitioner) {
+
+		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
+		FlinkKafkaProducer010<T> kafkaProducer = new FlinkKafkaProducer010<>(topicId, serializationSchema, producerConfig, customPartitioner);
+		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.10.x", objectTypeInfo, kafkaProducer);
+		return new FlinkKafkaProducer010Configuration<>(transformation, kafkaProducer);
+	}
+
+	// ---------------------- Regular constructors w/o timestamp support  ------------------
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param brokerList
+	 *			Comma separated addresses of the brokers
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined (keyless) serialization schema.
+	 */
+	public FlinkKafkaProducer010(String brokerList, String topicId, SerializationSchema<T> serializationSchema) {
+		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner<T>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined (keyless) serialization schema.
+	 * @param producerConfig
+	 * 			Properties with the producer configuration.
+	 */
+	public FlinkKafkaProducer010(String topicId, SerializationSchema<T> serializationSchema, Properties producerConfig) {
+		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner<T>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param topicId The topic to write data to
+	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
+	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
+	 */
+	public FlinkKafkaProducer010(String topicId, SerializationSchema<T> serializationSchema, Properties producerConfig, KafkaPartitioner<T> customPartitioner) {
+		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
+	}
+
+	// ------------------- Key/Value serialization schema constructors ----------------------
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param brokerList
+	 *			Comma separated addresses of the brokers
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined serialization schema supporting key/value messages
+	 */
+	public FlinkKafkaProducer010(String brokerList, String topicId, KeyedSerializationSchema<T> serializationSchema) {
+		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner<T>());
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+	 * the topic.
+	 *
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined serialization schema supporting key/value messages
+	 * @param producerConfig
+	 * 			Properties with the producer configuration.
+	 */
+	public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema<T> serializationSchema, Properties producerConfig) {
+		this(topicId, serializationSchema, producerConfig, new FixedPartitioner<T>());
+	}
+
+	/**
+	 * Create Kafka producer
+	 *
+	 * This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
+	 */
+	public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema<T> serializationSchema, Properties producerConfig, KafkaPartitioner<T> customPartitioner) {
+		// We create a Kafka 09 producer instance here and only "override" (by intercepting) the
+		// invoke call.
+		super(new FlinkKafkaProducer09<>(topicId, serializationSchema, producerConfig, customPartitioner));
+	}
+
+
+	// ----------------------------- Generic element processing  ---------------------------
+
+	private void invokeInternal(T next, long elementTimestamp) throws Exception {
+
+		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
+
+		internalProducer.checkErroneous();
+
+		byte[] serializedKey = internalProducer.schema.serializeKey(next);
+		byte[] serializedValue = internalProducer.schema.serializeValue(next);
+		String targetTopic = internalProducer.schema.getTargetTopic(next);
+		if (targetTopic == null) {
+			targetTopic = internalProducer.defaultTopicId;
+		}
+
+		Long timestamp = null;
+		if(this.writeTimestampToKafka) {
+			timestamp = elementTimestamp;
+		}
+
+		ProducerRecord<byte[], byte[]> record;
+		if (internalProducer.partitioner == null) {
+			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
+		} else {
+			record = new ProducerRecord<>(targetTopic, internalProducer.partitioner.partition(next, serializedKey, serializedValue, internalProducer.partitions.length), timestamp, serializedKey, serializedValue);
+		}
+		if (internalProducer.flushOnCheckpoint) {
+			synchronized (internalProducer.pendingRecordsLock) {
+				internalProducer.pendingRecords++;
+			}
+		}
+		internalProducer.producer.send(record, internalProducer.callback);
+	}
+
+
+	// ----------------- Helper methods implementing methods from SinkFunction and RichFunction (Approach (a)) ----
+
+
+	// ---- Configuration setters
+
+	/**
+	 * Defines whether the producer should fail on errors, or only log them.
+	 * If this is set to true, then exceptions will be only logged, if set to false,
+	 * exceptions will be eventually thrown and cause the streaming program to
+	 * fail (and enter recovery).
+	 *
+	 * Method is only accessible for approach (a) (see above)
+	 *
+	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
+	 */
+	public void setLogFailuresOnly(boolean logFailuresOnly) {
+		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
+		internalProducer.setLogFailuresOnly(logFailuresOnly);
+	}
+
+	/**
+	 * If set to true, the Flink producer will wait for all outstanding messages in the Kafka buffers
+	 * to be acknowledged by the Kafka producer on a checkpoint.
+	 * This way, the producer can guarantee that messages in the Kafka buffers are part of the checkpoint.
+	 *
+	 * Method is only accessible for approach (a) (see above)
+	 *
+	 * @param flush Flag indicating the flushing mode (true = flush on checkpoint)
+	 */
+	public void setFlushOnCheckpoint(boolean flush) {
+		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
+		internalProducer.setFlushOnCheckpoint(flush);
+	}
+
+	/**
+	 * This method is used for approach (a) (see above)
+	 *
+	 */
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
+		internalProducer.open(parameters);
+	}
+
+	/**
+	 * This method is used for approach (a) (see above)
+	 */
+	@Override
+	public IterationRuntimeContext getIterationRuntimeContext() {
+		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
+		return internalProducer.getIterationRuntimeContext();
+	}
+
+	/**
+	 * This method is used for approach (a) (see above)
+	 */
+	@Override
+	public void setRuntimeContext(RuntimeContext t) {
+		final FlinkKafkaProducerBase<T> internalProducer = (FlinkKafkaProducerBase<T>) userFunction;
+		internalProducer.setRuntimeContext(t);
+	}
+
+	/**
+	 * Invoke method for using the Sink as DataStream.addSink() sink.
+	 *
+	 * This method is used for approach (a) (see above)
+	 *
+	 * @param value The input record.
+	 */
+	@Override
+	public void invoke(T value) throws Exception {
+		invokeInternal(value, Long.MAX_VALUE);
+	}
+
+
+	// ----------------- Helper methods and classes implementing methods from StreamSink (Approach (b)) ----
+
+
+	/**
+	 * Process method for using the sink with timestamp support.
+	 *
+	 * This method is used for approach (b) (see above)
+	 */
+	@Override
+	public void processElement(StreamRecord<T> element) throws Exception {
+		invokeInternal(element.getValue(), element.getTimestamp());
+	}
+
+	/**
+	 * Configuration object returned by the writeToKafkaWithTimestamps() call.
+	 */
+	public static class FlinkKafkaProducer010Configuration<T> extends DataStreamSink<T> {
+
+		private final FlinkKafkaProducerBase wrappedProducerBase;
+		private final FlinkKafkaProducer010 producer;
+
+		private FlinkKafkaProducer010Configuration(DataStream stream, FlinkKafkaProducer010<T> producer) {
+			//noinspection unchecked
+			super(stream, producer);
+			this.producer = producer;
+			this.wrappedProducerBase = (FlinkKafkaProducerBase) producer.userFunction;
+		}
+
+		/**
+		 * Defines whether the producer should fail on errors, or only log them.
+		 * If this is set to true, then exceptions will be only logged, if set to false,
+		 * exceptions will be eventually thrown and cause the streaming program to
+		 * fail (and enter recovery).
+		 *
+		 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
+		 */
+		public void setLogFailuresOnly(boolean logFailuresOnly) {
+			this.wrappedProducerBase.setLogFailuresOnly(logFailuresOnly);
+		}
+
+		/**
+		 * If set to true, the Flink producer will wait for all outstanding messages in the Kafka buffers
+		 * to be acknowledged by the Kafka producer on a checkpoint.
+		 * This way, the producer can guarantee that messages in the Kafka buffers are part of the checkpoint.
+		 *
+		 * @param flush Flag indicating the flushing mode (true = flush on checkpoint)
+		 */
+		public void setFlushOnCheckpoint(boolean flush) {
+			this.wrappedProducerBase.setFlushOnCheckpoint(flush);
+		}
+
+		/**
+		 * If set to true, Flink will write the (event time) timestamp attached to each record into Kafka.
+		 * Timestamps must be positive for Kafka to accept them.
+		 *
+		 * @param writeTimestampToKafka Flag indicating if Flink's internal timestamps are written to Kafka.
+		 */
+		public void setWriteTimestampToKafka(boolean writeTimestampToKafka) {
+			this.producer.writeTimestampToKafka = writeTimestampToKafka;
+		}
+	}
+
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
new file mode 100644
index 0000000..ddf1ad3
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.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;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Kafka {@link StreamTableSource} for Kafka 0.10.
+ */
+public class Kafka010JsonTableSource extends Kafka09JsonTableSource {
+
+	/**
+	 * Creates a Kafka 0.10 JSON {@link StreamTableSource}.
+	 *
+	 * @param topic      Kafka topic to consume.
+	 * @param properties Properties for the Kafka consumer.
+	 * @param fieldNames Row field names.
+	 * @param fieldTypes Row field types.
+	 */
+	public Kafka010JsonTableSource(
+			String topic,
+			Properties properties,
+			String[] fieldNames,
+			TypeInformation<?>[] fieldTypes) {
+
+		super(topic, properties, fieldNames, fieldTypes);
+	}
+
+	/**
+	 * Creates a Kafka 0.10 JSON {@link StreamTableSource}.
+	 *
+	 * @param topic      Kafka topic to consume.
+	 * @param properties Properties for the Kafka consumer.
+	 * @param fieldNames Row field names.
+	 * @param fieldTypes Row field types.
+	 */
+	public Kafka010JsonTableSource(
+			String topic,
+			Properties properties,
+			String[] fieldNames,
+			Class<?>[] fieldTypes) {
+
+		super(topic, properties, fieldNames, fieldTypes);
+	}
+
+	@Override
+	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
+		return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
new file mode 100644
index 0000000..732440b
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Kafka {@link StreamTableSource} for Kafka 0.10.
+ */
+public class Kafka010TableSource extends Kafka09TableSource {
+
+	/**
+	 * Creates a Kafka 0.10 {@link StreamTableSource}.
+	 *
+	 * @param topic                 Kafka topic to consume.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param deserializationSchema Deserialization schema to use for Kafka records.
+	 * @param fieldNames            Row field names.
+	 * @param fieldTypes            Row field types.
+	 */
+	public Kafka010TableSource(
+			String topic,
+			Properties properties,
+			DeserializationSchema<Row> deserializationSchema,
+			String[] fieldNames,
+			TypeInformation<?>[] fieldTypes) {
+
+		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
+	}
+
+	/**
+	 * Creates a Kafka 0.10 {@link StreamTableSource}.
+	 *
+	 * @param topic                 Kafka topic to consume.
+	 * @param properties            Properties for the Kafka consumer.
+	 * @param deserializationSchema Deserialization schema to use for Kafka records.
+	 * @param fieldNames            Row field names.
+	 * @param fieldTypes            Row field types.
+	 */
+	public Kafka010TableSource(
+			String topic,
+			Properties properties,
+			DeserializationSchema<Row> deserializationSchema,
+			String[] fieldNames,
+			Class<?>[] fieldTypes) {
+
+		super(topic, properties, deserializationSchema, fieldNames, fieldTypes);
+	}
+
+	@Override
+	FlinkKafkaConsumerBase<Row> getKafkaConsumer(String topic, Properties properties, DeserializationSchema<Row> deserializationSchema) {
+		return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java
new file mode 100644
index 0000000..71dd29a
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internal;
+
+import org.apache.flink.metrics.MetricGroup;
+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.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.SerializedValue;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.TopicPartition;
+
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * A fetcher that fetches data from Kafka brokers via the Kafka 0.10 consumer API.
+ * 
+ * <p>This fetcher re-uses basically all functionality of the 0.9 fetcher. It only additionally
+ * takes the KafkaRecord-attached timestamp and attaches it to the Flink records.
+ * 
+ * @param <T> The type of elements produced by the fetcher.
+ */
+public class Kafka010Fetcher<T> extends Kafka09Fetcher<T> {
+
+	public Kafka010Fetcher(
+			SourceContext<T> sourceContext,
+			List<KafkaTopicPartition> assignedPartitions,
+			SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+			SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+			ProcessingTimeService processingTimeProvider,
+			long autoWatermarkInterval,
+			ClassLoader userCodeClassLoader,
+			boolean enableCheckpointing,
+			String taskNameWithSubtasks,
+			MetricGroup metricGroup,
+			KeyedDeserializationSchema<T> deserializer,
+			Properties kafkaProperties,
+			long pollTimeout,
+			boolean useMetrics) throws Exception
+	{
+		super(
+				sourceContext,
+				assignedPartitions,
+				watermarksPeriodic,
+				watermarksPunctuated,
+				processingTimeProvider,
+				autoWatermarkInterval,
+				userCodeClassLoader,
+				enableCheckpointing,
+				taskNameWithSubtasks,
+				metricGroup,
+				deserializer,
+				kafkaProperties,
+				pollTimeout,
+				useMetrics);
+	}
+
+	@Override
+	protected void emitRecord(
+			T record,
+			KafkaTopicPartitionState<TopicPartition> partition,
+			long offset,
+			ConsumerRecord<?, ?> consumerRecord) throws Exception {
+
+		// we attach the Kafka 0.10 timestamp here
+		emitRecordWithTimestamp(record, partition, offset, consumerRecord.timestamp());
+	}
+
+	/**
+	 * This method needs to be overridden because Kafka broke binary compatibility between 0.9 and 0.10,
+	 * changing binary signatures
+	 */
+	@Override
+	protected KafkaConsumerCallBridge010 createCallBridge() {
+		return new KafkaConsumerCallBridge010();
+	}
+
+	@Override
+	protected String getFetcherName() {
+		return "Kafka 0.10 Fetcher";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java
new file mode 100644
index 0000000..a81b098
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.TopicPartition;
+
+import java.util.List;
+
+/**
+ * The ConsumerCallBridge simply calls the {@link KafkaConsumer#assign(java.util.Collection)} method.
+ * 
+ * This indirection is necessary, because Kafka broke binary compatibility between 0.9 and 0.10,
+ * changing {@code assign(List)} to {@code assign(Collection)}.
+ * 
+ * Because of that, we need two versions whose compiled code goes against different method signatures.
+ */
+public class KafkaConsumerCallBridge010 extends KafkaConsumerCallBridge {
+
+	@Override
+	public void assignPartitions(KafkaConsumer<?, ?> consumer, List<TopicPartition> topicPartitions) throws Exception {
+		consumer.assign(topicPartitions);
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java
new file mode 100644
index 0000000..6ee0429
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java
@@ -0,0 +1,484 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.core.testutils.MultiShotLatch;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+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.connectors.kafka.internal.Handover;
+import org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher;
+import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+
+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.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.common.TopicPartition;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.mockito.Mockito;
+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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyLong;
+import static org.powermock.api.mockito.PowerMockito.doAnswer;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+/**
+ * Unit tests for the {@link Kafka010Fetcher}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(KafkaConsumerThread.class)
+public class Kafka010FetcherTest {
+
+    @Test
+    public void testCommitDoesNotBlock() throws Exception {
+
+        // test data
+        final KafkaTopicPartition testPartition = new KafkaTopicPartition("test", 42);
+        final Map<KafkaTopicPartition, Long> testCommitData = new HashMap<>();
+        testCommitData.put(testPartition, 11L);
+
+        // to synchronize when the consumer is in its blocking method
+        final OneShotLatch sync = new OneShotLatch();
+
+        // ----- the mock consumer with blocking poll calls ----
+        final MultiShotLatch blockerLatch = new MultiShotLatch();
+
+        KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
+        when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
+
+            @Override
+            public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) throws InterruptedException {
+                sync.trigger();
+                blockerLatch.await();
+                return ConsumerRecords.empty();
+            }
+        });
+
+        doAnswer(new Answer<Void>() {
+            @Override
+            public Void answer(InvocationOnMock invocation) {
+                blockerLatch.trigger();
+                return null;
+            }
+        }).when(mockConsumer).wakeup();
+
+        // make sure the fetcher creates the mock consumer
+        whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
+
+        // ----- create the test fetcher -----
+
+        @SuppressWarnings("unchecked")
+        SourceContext<String> sourceContext = mock(SourceContext.class);
+        List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition("test", 42));
+        KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
+
+        final Kafka010Fetcher<String> fetcher = new Kafka010Fetcher<>(
+                sourceContext,
+                topics,
+                null, /* periodic assigner */
+                null, /* punctuated assigner */
+                new TestProcessingTimeService(),
+                10,
+                getClass().getClassLoader(),
+                false, /* checkpointing */
+                "taskname-with-subtask",
+                new UnregisteredMetricsGroup(),
+                schema,
+                new Properties(),
+                0L,
+                false);
+
+        // ----- run the fetcher -----
+
+        final AtomicReference<Throwable> error = new AtomicReference<>();
+        final Thread fetcherRunner = new Thread("fetcher runner") {
+
+            @Override
+            public void run() {
+                try {
+                    fetcher.runFetchLoop();
+                } catch (Throwable t) {
+                    error.set(t);
+                }
+            }
+        };
+        fetcherRunner.start();
+
+        // wait until the fetcher has reached the method of interest
+        sync.await();
+
+        // ----- trigger the offset commit -----
+
+        final AtomicReference<Throwable> commitError = new AtomicReference<>();
+        final Thread committer = new Thread("committer runner") {
+            @Override
+            public void run() {
+                try {
+                    fetcher.commitInternalOffsetsToKafka(testCommitData);
+                } catch (Throwable t) {
+                    commitError.set(t);
+                }
+            }
+        };
+        committer.start();
+
+        // ----- ensure that the committer finishes in time  -----
+        committer.join(30000);
+        assertFalse("The committer did not finish in time", committer.isAlive());
+
+        // ----- test done, wait till the fetcher is done for a clean shutdown -----
+        fetcher.cancel();
+        fetcherRunner.join();
+
+        // check that there were no errors in the fetcher
+        final Throwable fetcherError = error.get();
+        if (fetcherError != null && !(fetcherError instanceof Handover.ClosedException)) {
+            throw new Exception("Exception in the fetcher", fetcherError);
+        }
+        final Throwable committerError = commitError.get();
+        if (committerError != null) {
+            throw new Exception("Exception in the committer", committerError);
+        }
+    }
+
+    @Test
+    public void ensureOffsetsGetCommitted() throws Exception {
+
+        // test data
+        final KafkaTopicPartition testPartition1 = new KafkaTopicPartition("test", 42);
+        final KafkaTopicPartition testPartition2 = new KafkaTopicPartition("another", 99);
+
+        final Map<KafkaTopicPartition, Long> testCommitData1 = new HashMap<>();
+        testCommitData1.put(testPartition1, 11L);
+        testCommitData1.put(testPartition2, 18L);
+
+        final Map<KafkaTopicPartition, Long> testCommitData2 = new HashMap<>();
+        testCommitData2.put(testPartition1, 19L);
+        testCommitData2.put(testPartition2, 28L);
+
+        final BlockingQueue<Map<TopicPartition, OffsetAndMetadata>> commitStore = new LinkedBlockingQueue<>();
+
+
+        // ----- the mock consumer with poll(), wakeup(), and commit(A)sync calls ----
+
+        final MultiShotLatch blockerLatch = new MultiShotLatch();
+
+        KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
+
+        when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
+            @Override
+            public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) throws InterruptedException {
+                blockerLatch.await();
+                return ConsumerRecords.empty();
+            }
+        });
+
+        doAnswer(new Answer<Void>() {
+            @Override
+            public Void answer(InvocationOnMock invocation) {
+                blockerLatch.trigger();
+                return null;
+            }
+        }).when(mockConsumer).wakeup();
+
+        doAnswer(new Answer<Void>() {
+            @Override
+            public Void answer(InvocationOnMock invocation) {
+                @SuppressWarnings("unchecked")
+                Map<TopicPartition, OffsetAndMetadata> offsets =
+                        (Map<TopicPartition, OffsetAndMetadata>) invocation.getArguments()[0];
+
+                OffsetCommitCallback callback = (OffsetCommitCallback) invocation.getArguments()[1];
+
+                commitStore.add(offsets);
+                callback.onComplete(offsets, null);
+
+                return null;
+            }
+        }).when(mockConsumer).commitAsync(
+                Mockito.<Map<TopicPartition, OffsetAndMetadata>>any(), any(OffsetCommitCallback.class));
+
+        // make sure the fetcher creates the mock consumer
+        whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
+
+        // ----- create the test fetcher -----
+
+        @SuppressWarnings("unchecked")
+        SourceContext<String> sourceContext = mock(SourceContext.class);
+        List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition("test", 42));
+        KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
+        StreamingRuntimeContext context = mock(StreamingRuntimeContext.class);
+
+        final Kafka010Fetcher<String> fetcher = new Kafka010Fetcher<>(
+                sourceContext,
+                topics,
+                null, /* periodic assigner */
+                null, /* punctuated assigner */
+                new TestProcessingTimeService(),
+                10,
+                getClass().getClassLoader(),
+                false, /* checkpointing */
+                "taskname-with-subtask",
+                new UnregisteredMetricsGroup(),
+                schema,
+                new Properties(),
+                0L,
+                false);
+
+
+        // ----- run the fetcher -----
+
+        final AtomicReference<Throwable> error = new AtomicReference<>();
+        final Thread fetcherRunner = new Thread("fetcher runner") {
+
+            @Override
+            public void run() {
+                try {
+                    fetcher.runFetchLoop();
+                } catch (Throwable t) {
+                    error.set(t);
+                }
+            }
+        };
+        fetcherRunner.start();
+
+        // ----- trigger the first offset commit -----
+
+        fetcher.commitInternalOffsetsToKafka(testCommitData1);
+        Map<TopicPartition, OffsetAndMetadata> result1 = commitStore.take();
+
+        for (Entry<TopicPartition, OffsetAndMetadata> entry : result1.entrySet()) {
+            TopicPartition partition = entry.getKey();
+            if (partition.topic().equals("test")) {
+                assertEquals(42, partition.partition());
+                assertEquals(12L, entry.getValue().offset());
+            }
+            else if (partition.topic().equals("another")) {
+                assertEquals(99, partition.partition());
+                assertEquals(18L, entry.getValue().offset());
+            }
+        }
+
+        // ----- trigger the second offset commit -----
+
+        fetcher.commitInternalOffsetsToKafka(testCommitData2);
+        Map<TopicPartition, OffsetAndMetadata> result2 = commitStore.take();
+
+        for (Entry<TopicPartition, OffsetAndMetadata> entry : result2.entrySet()) {
+            TopicPartition partition = entry.getKey();
+            if (partition.topic().equals("test")) {
+                assertEquals(42, partition.partition());
+                assertEquals(20L, entry.getValue().offset());
+            }
+            else if (partition.topic().equals("another")) {
+                assertEquals(99, partition.partition());
+                assertEquals(28L, entry.getValue().offset());
+            }
+        }
+
+        // ----- test done, wait till the fetcher is done for a clean shutdown -----
+        fetcher.cancel();
+        fetcherRunner.join();
+
+        // check that there were no errors in the fetcher
+        final Throwable caughtError = error.get();
+        if (caughtError != null && !(caughtError instanceof Handover.ClosedException)) {
+            throw new Exception("Exception in the fetcher", caughtError);
+        }
+    }
+
+    @Test
+    public void testCancellationWhenEmitBlocks() throws Exception {
+
+        // ----- some test data -----
+
+        final String topic = "test-topic";
+        final int partition = 3;
+        final byte[] payload = new byte[] {1, 2, 3, 4};
+
+        final List<ConsumerRecord<byte[], byte[]>> records = Arrays.asList(
+                new ConsumerRecord<byte[], byte[]>(topic, partition, 15, payload, payload),
+                new ConsumerRecord<byte[], byte[]>(topic, partition, 16, payload, payload),
+                new ConsumerRecord<byte[], byte[]>(topic, partition, 17, payload, payload));
+
+        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> data = new HashMap<>();
+        data.put(new TopicPartition(topic, partition), records);
+
+        final ConsumerRecords<byte[], byte[]> consumerRecords = new ConsumerRecords<>(data);
+
+        // ----- the test consumer -----
+
+        final KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
+        when(mockConsumer.poll(anyLong())).thenAnswer(new Answer<ConsumerRecords<?, ?>>() {
+            @Override
+            public ConsumerRecords<?, ?> answer(InvocationOnMock invocation) {
+                return consumerRecords;
+            }
+        });
+
+        whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
+
+        // ----- build a fetcher -----
+
+        BlockingSourceContext<String> sourceContext = new BlockingSourceContext<>();
+        List<KafkaTopicPartition> topics = Collections.singletonList(new KafkaTopicPartition(topic, partition));
+        KeyedDeserializationSchema<String> schema = new KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
+
+        final Kafka010Fetcher<String> fetcher = new Kafka010Fetcher<>(
+                sourceContext,
+                topics,
+                null, /* periodic watermark extractor */
+                null, /* punctuated watermark extractor */
+                new TestProcessingTimeService(),
+                10, /* watermark interval */
+                this.getClass().getClassLoader(),
+                true, /* checkpointing */
+                "task_name",
+                new UnregisteredMetricsGroup(),
+                schema,
+                new Properties(),
+                0L,
+                false);
+
+
+        // ----- run the fetcher -----
+
+        final AtomicReference<Throwable> error = new AtomicReference<>();
+        final Thread fetcherRunner = new Thread("fetcher runner") {
+
+            @Override
+            public void run() {
+                try {
+                    fetcher.runFetchLoop();
+                } catch (Throwable t) {
+                    error.set(t);
+                }
+            }
+        };
+        fetcherRunner.start();
+
+        // wait until the thread started to emit records to the source context
+        sourceContext.waitTillHasBlocker();
+
+        // now we try to cancel the fetcher, including the interruption usually done on the task thread
+        // once it has finished, there must be no more thread blocked on the source context
+        fetcher.cancel();
+        fetcherRunner.interrupt();
+        fetcherRunner.join();
+
+        assertFalse("fetcher threads did not properly finish", sourceContext.isStillBlocking());
+    }
+
+    // ------------------------------------------------------------------------
+    //  test utilities
+    // ------------------------------------------------------------------------
+
+    private static final class BlockingSourceContext<T> implements SourceContext<T> {
+
+        private final ReentrantLock lock = new ReentrantLock();
+        private final OneShotLatch inBlocking = new OneShotLatch();
+
+        @Override
+        public void collect(T element) {
+            block();
+        }
+
+        @Override
+        public void collectWithTimestamp(T element, long timestamp) {
+            block();
+        }
+
+        @Override
+        public void emitWatermark(Watermark mark) {
+            block();
+        }
+
+        @Override
+        public Object getCheckpointLock() {
+            return new Object();
+        }
+
+        @Override
+        public void close() {}
+
+        public void waitTillHasBlocker() throws InterruptedException {
+            inBlocking.await();
+        }
+
+        public boolean isStillBlocking() {
+            return lock.isLocked();
+        }
+
+        @SuppressWarnings({"InfiniteLoopStatement", "SynchronizationOnLocalVariableOrMethodParameter"})
+        private void block() {
+            lock.lock();
+            try {
+                inBlocking.trigger();
+
+                // put this thread to sleep indefinitely
+                final Object o = new Object();
+                while (true) {
+                    synchronized (o) {
+                        o.wait();
+                    }
+                }
+            }
+            catch (InterruptedException e) {
+                // exit cleanly, simply reset the interruption flag
+                Thread.currentThread().interrupt();
+            }
+            finally {
+                lock.unlock();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
new file mode 100644
index 0000000..08511c9
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+
+
+public class Kafka010ITCase extends KafkaConsumerTestBase {
+
+	// ------------------------------------------------------------------------
+	//  Suite of Tests
+	// ------------------------------------------------------------------------
+
+
+	@Test(timeout = 60000)
+	public void testFailOnNoBroker() throws Exception {
+		runFailOnNoBrokerTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testConcurrentProducerConsumerTopology() throws Exception {
+		runSimpleConcurrentProducerConsumerTopology();
+	}
+
+	@Test(timeout = 60000)
+	public void testKeyValueSupport() throws Exception {
+		runKeyValueTest();
+	}
+
+	// --- canceling / failures ---
+
+	@Test(timeout = 60000)
+	public void testCancelingEmptyTopic() throws Exception {
+		runCancelingOnEmptyInputTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testCancelingFullTopic() throws Exception {
+		runCancelingOnFullInputTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testFailOnDeploy() throws Exception {
+		runFailOnDeployTest();
+	}
+
+
+	// --- source to partition mappings and exactly once ---
+
+	@Test(timeout = 60000)
+	public void testOneToOneSources() throws Exception {
+		runOneToOneExactlyOnceTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testOneSourceMultiplePartitions() throws Exception {
+		runOneSourceMultiplePartitionsExactlyOnceTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testMultipleSourcesOnePartition() throws Exception {
+		runMultipleSourcesOnePartitionExactlyOnceTest();
+	}
+
+	// --- broker failure ---
+
+	@Test(timeout = 60000)
+	public void testBrokerFailure() throws Exception {
+		runBrokerFailureTest();
+	}
+
+	// --- special executions ---
+
+	@Test(timeout = 60000)
+	public void testBigRecordJob() throws Exception {
+		runBigRecordTestTopology();
+	}
+
+	@Test(timeout = 60000)
+	public void testMultipleTopics() throws Exception {
+		runProduceConsumeMultipleTopics();
+	}
+
+	@Test(timeout = 60000)
+	public void testAllDeletes() throws Exception {
+		runAllDeletesTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testMetricsAndEndOfStream() throws Exception {
+		runEndOfStreamTest();
+	}
+
+	// --- offset committing ---
+
+	@Test(timeout = 60000)
+	public void testCommitOffsetsToKafka() throws Exception {
+		runCommitOffsetsToKafka();
+	}
+
+	@Test(timeout = 60000)
+	public void testStartFromKafkaCommitOffsets() throws Exception {
+		runStartFromKafkaCommitOffsets();
+	}
+
+	@Test(timeout = 60000)
+	public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception {
+		runAutoOffsetRetrievalAndCommitToKafka();
+	}
+
+	/**
+	 * Kafka 0.10 specific test, ensuring Timestamps are properly written to and read from Kafka
+	 */
+	@Test(timeout = 60000)
+	public void testTimestamps() throws Exception {
+
+		final String topic = "tstopic";
+		createTestTopic(topic, 3, 1);
+
+		// ---------- Produce an event time stream into Kafka -------------------
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(1);
+		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+		env.getConfig().disableSysoutLogging();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+		DataStream<Long> streamWithTimestamps = env.addSource(new SourceFunction<Long>() {
+			boolean running = true;
+
+			@Override
+			public void run(SourceContext<Long> ctx) throws Exception {
+				long i = 0;
+				while(running) {
+					ctx.collectWithTimestamp(i, i*2);
+					if(i++ == 1000L) {
+						running = false;
+					}
+				}
+			}
+
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		});
+
+		final TypeInformationSerializationSchema<Long> longSer = new TypeInformationSerializationSchema<>(TypeInfoParser.<Long>parse("Long"), env.getConfig());
+		FlinkKafkaProducer010.FlinkKafkaProducer010Configuration prod = FlinkKafkaProducer010.writeToKafkaWithTimestamps(streamWithTimestamps, topic, new KeyedSerializationSchemaWrapper<>(longSer), standardProps, new KafkaPartitioner<Long>() {
+			@Override
+			public int partition(Long next, byte[] serializedKey, byte[] serializedValue, int numPartitions) {
+				return (int)(next % 3);
+			}
+		});
+		prod.setParallelism(3);
+		prod.setWriteTimestampToKafka(true);
+		env.execute("Produce some");
+
+		// ---------- Consume stream from Kafka -------------------
+
+		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(1);
+		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+		env.getConfig().disableSysoutLogging();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+		FlinkKafkaConsumer010<Long> kafkaSource = new FlinkKafkaConsumer010<>(topic, new LimitedLongDeserializer(), standardProps);
+		kafkaSource.assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks<Long>() {
+			@Nullable
+			@Override
+			public Watermark checkAndGetNextWatermark(Long lastElement, long extractedTimestamp) {
+				if(lastElement % 10 == 0) {
+					return new Watermark(lastElement);
+				}
+				return null;
+			}
+
+			@Override
+			public long extractTimestamp(Long element, long previousElementTimestamp) {
+				return previousElementTimestamp;
+			}
+		});
+
+		DataStream<Long> stream = env.addSource(kafkaSource);
+		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
+		stream.transform("timestamp validating operator", objectTypeInfo, new TimestampValidatingOperator()).setParallelism(1);
+
+		env.execute("Consume again");
+
+		deleteTestTopic(topic);
+	}
+
+	private static class TimestampValidatingOperator extends StreamSink<Long> {
+
+		public TimestampValidatingOperator() {
+			super(new SinkFunction<Long>() {
+				@Override
+				public void invoke(Long value) throws Exception {
+					throw new RuntimeException("Unexpected");
+				}
+			});
+		}
+
+		long elCount = 0;
+		long wmCount = 0;
+		long lastWM = Long.MIN_VALUE;
+
+		@Override
+		public void processElement(StreamRecord<Long> element) throws Exception {
+			elCount++;
+			if(element.getValue() * 2 != element.getTimestamp()) {
+				throw new RuntimeException("Invalid timestamp: " + element);
+			}
+		}
+
+		@Override
+		public void processWatermark(Watermark mark) throws Exception {
+			wmCount++;
+
+			if(lastWM <= mark.getTimestamp()) {
+				lastWM = mark.getTimestamp();
+			} else {
+				throw new RuntimeException("Received watermark higher than the last one");
+			}
+
+			if( mark.getTimestamp() % 10 != 0 && mark.getTimestamp() != Long.MAX_VALUE ) {
+				throw new RuntimeException("Invalid watermark: " + mark.getTimestamp());
+			}
+		}
+
+		@Override
+		public void close() throws Exception {
+			super.close();
+			if(elCount != 1000L) {
+				throw new RuntimeException("Wrong final element count " + elCount);
+			}
+
+			if(wmCount <= 2) {
+				throw new RuntimeException("Almost no watermarks have been sent " + wmCount);
+			}
+		}
+	}
+
+	private static class LimitedLongDeserializer implements KeyedDeserializationSchema<Long> {
+
+		private final TypeInformation<Long> ti;
+		private final TypeSerializer<Long> ser;
+		long cnt = 0;
+
+		public LimitedLongDeserializer() {
+			this.ti = TypeInfoParser.parse("Long");
+			this.ser = ti.createSerializer(new ExecutionConfig());
+		}
+		@Override
+		public TypeInformation<Long> getProducedType() {
+			return ti;
+		}
+
+		@Override
+		public Long deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException {
+			cnt++;
+			DataInputView in = new DataInputViewStreamWrapper(new ByteArrayInputStream(message));
+			Long e = ser.deserialize(in);
+			return e;
+		}
+
+		@Override
+		public boolean isEndOfStream(Long nextElement) {
+			return cnt > 1000L;
+		}
+	}
+
+}


[36/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java
new file mode 100644
index 0000000..d015157
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java
@@ -0,0 +1,481 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.metrics.MetricGroup;
+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 subtask's runtime context */
+	private final RuntimeContext runtimeContext;
+
+	/** 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,
+			boolean useMetrics) throws Exception
+	{
+		super(
+				sourceContext,
+				assignedPartitions,
+				watermarksPeriodic,
+				watermarksPunctuated,
+				runtimeContext.getProcessingTimeService(),
+				runtimeContext.getExecutionConfig().getAutoWatermarkInterval(),
+				runtimeContext.getUserCodeClassLoader(),
+				useMetrics);
+
+		this.deserializer = checkNotNull(deserializer);
+		this.kafkaConfig = checkNotNull(kafkaProperties);
+		this.runtimeContext = runtimeContext;
+		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.getCommittedOffsets(partitionsWithNoOffset);
+				for (KafkaTopicPartitionState<TopicAndPartition> partition : subscribedPartitions()) {
+					Long zkOffset = zkOffsets.get(partition.getKafkaTopicPartition());
+					if (zkOffset != null) {
+						// the offset in ZK represents the "next record to process", so we need to subtract it by 1
+						// to correctly represent our internally checkpointed offsets
+						partition.setOffset(zkOffset - 1);
+					}
+				}
+			}
+
+			// 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();
+			}
+
+			// register offset metrics
+			if (useMetrics) {
+				final MetricGroup kafkaMetricGroup = runtimeContext.getMetricGroup().addGroup("KafkaConsumer");
+				addOffsetStateGauge(kafkaMetricGroup);
+			}
+
+			// 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();
+			}
+
+			// clear the interruption flag
+			// this allows the joining on consumer threads (on best effort) to happen in
+			// case the initial interrupt already
+			Thread.interrupted();
+
+			// 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 (InterruptedException ignored) {
+				// waiting for the thread shutdown apparently got interrupted
+				// restore interrupted state and continue
+				Thread.currentThread().interrupt();
+			}
+			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 commitInternalOffsetsToKafka(Map<KafkaTopicPartition, Long> offsets) throws Exception {
+		ZookeeperOffsetHandler zkHandler = this.zookeeperOffsetHandler;
+		if (zkHandler != null) {
+			// the ZK handler takes care of incrementing the offsets by 1 before committing
+			zkHandler.prepareAndCommitOffsets(offsets);
+		}
+
+		// Set committed offsets in topic partition state
+		KafkaTopicPartitionState<TopicAndPartition>[] partitions = subscribedPartitions();
+		for (KafkaTopicPartitionState<TopicAndPartition> partition : partitions) {
+			Long offset = offsets.get(partition.getKafkaTopicPartition());
+			if (offset != null) {
+				partition.setCommittedOffset(offset);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  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, runtimeContext.getUserCodeClassLoader());
+
+		// 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)",
+				runtimeContext.getTaskName(), 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/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java b/flink-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-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/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java b/flink-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-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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java
new file mode 100644
index 0000000..27d90f2
--- /dev/null
+++ b/flink-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> offsetsToCommit = new HashMap<>(partitionStates.length);
+				for (KafkaTopicPartitionState<?> partitionState : partitionStates) {
+					offsetsToCommit.put(partitionState.getKafkaTopicPartition(), partitionState.getOffset());
+				}
+				
+				offsetHandler.prepareAndCommitOffsets(offsetsToCommit);
+			}
+		}
+		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/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java
new file mode 100644
index 0000000..35e491a
--- /dev/null
+++ b/flink-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.util.PropertiesUtil.getInt;
+
+/**
+ * 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 = getInt(config, "socket.timeout.ms", 30000);
+		this.minBytes = getInt(config, "fetch.min.bytes", 1);
+		this.maxWait = getInt(config, "fetch.wait.max.ms", 100);
+		this.fetchSize = getInt(config, "fetch.message.max.bytes", 1048576);
+		this.bufferSize = getInt(config, "socket.receive.buffer.bytes", 65536);
+		this.reconnectLimit = getInt(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/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
new file mode 100644
index 0000000..8f2ef09
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.utils.ZKGroupTopicDirs;
+
+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.kafka.clients.consumer.ConsumerConfig;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Handler for committing Kafka offsets to Zookeeper and to retrieve them again.
+ */
+public class ZookeeperOffsetHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZookeeperOffsetHandler.class);
+
+	private final String groupId;
+
+	private final CuratorFramework curatorClient;
+
+
+	public ZookeeperOffsetHandler(Properties props) {
+		this.groupId = props.getProperty(ConsumerConfig.GROUP_ID_CONFIG);
+		if (this.groupId == null) {
+			throw new IllegalArgumentException("Required property '"
+					+ ConsumerConfig.GROUP_ID_CONFIG + "' has not been set");
+		}
+		
+		String zkConnect = props.getProperty("zookeeper.connect");
+		if (zkConnect == null) {
+			throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set");
+		}
+
+		// we use Curator's default timeouts
+		int sessionTimeoutMs =  Integer.valueOf(props.getProperty("zookeeper.session.timeout.ms", "60000"));
+		int connectionTimeoutMs = Integer.valueOf(props.getProperty("zookeeper.connection.timeout.ms", "15000"));
+		
+		// undocumented config options allowing users to configure the retry policy. (they are "flink." prefixed as they are no official kafka configs)
+		int backoffBaseSleepTime = Integer.valueOf(props.getProperty("flink.zookeeper.base-sleep-time.ms", "100"));
+		int backoffMaxRetries =  Integer.valueOf(props.getProperty("flink.zookeeper.max-retries", "10"));
+		
+		RetryPolicy retryPolicy = new ExponentialBackoffRetry(backoffBaseSleepTime, backoffMaxRetries);
+		curatorClient = CuratorFrameworkFactory.newClient(zkConnect, sessionTimeoutMs, connectionTimeoutMs, retryPolicy);
+		curatorClient.start();
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Offset access and manipulation
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Commits offsets for Kafka partitions to ZooKeeper. The given offsets to this method should be the offsets of
+	 * the last processed records; this method will take care of incrementing the offsets by 1 before committing them so
+	 * that the committed offsets to Zookeeper represent the next record to process.
+	 * 
+	 * @param internalOffsets The internal offsets (representing last processed records) for the partitions to commit.
+	 * @throws Exception The method forwards exceptions.
+	 */
+	public void prepareAndCommitOffsets(Map<KafkaTopicPartition, Long> internalOffsets) throws Exception {
+		for (Map.Entry<KafkaTopicPartition, Long> entry : internalOffsets.entrySet()) {
+			KafkaTopicPartition tp = entry.getKey();
+
+			Long lastProcessedOffset = entry.getValue();
+			if (lastProcessedOffset != null && lastProcessedOffset >= 0) {
+				setOffsetInZooKeeper(curatorClient, groupId, tp.getTopic(), tp.getPartition(), lastProcessedOffset + 1);
+			}
+		}
+	}
+
+	/**
+	 * @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> getCommittedOffsets(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());
+
+			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);
+			}
+		}
+		return ret;
+	}
+
+	/**
+	 * Closes the offset handler.
+	 * 
+	 * @throws IOException Thrown, if the handler cannot be closed properly.
+	 */
+	public void close() throws IOException {
+		curatorClient.close();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Communication with Zookeeper
+	// ------------------------------------------------------------------------
+	
+	public static void setOffsetInZooKeeper(CuratorFramework curatorClient, String groupId, String topic, int partition, long offset) throws Exception {
+		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, topic);
+		String path = topicDirs.consumerOffsetDir() + "/" + partition;
+		curatorClient.newNamespaceAwareEnsurePath(path).ensure(curatorClient.getZookeeperClient());
+		byte[] data = Long.toString(offset).getBytes();
+		curatorClient.setData().forPath(path, data);
+	}
+
+	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());
+		
+		byte[] data = curatorClient.getData().forPath(path);
+		
+		if (data == null) {
+			return null;
+		} else {
+			String asString = new String(data);
+			if (asString.length() == 0) {
+				return null;
+			} else {
+				try {
+					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/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
new file mode 100644
index 0000000..fabb0fe
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.curator.framework.CuratorFramework;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.runtime.client.JobCancellationException;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
+import org.apache.flink.streaming.connectors.kafka.testutils.JobManagerCommunicationUtils;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Properties;
+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 {
+
+	// ------------------------------------------------------------------------
+	//  Suite of Tests
+	// ------------------------------------------------------------------------
+
+	@Test(timeout = 60000)
+	public void testFailOnNoBroker() throws Exception {
+		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();
+	}
+
+	// --- canceling / failures ---
+
+	@Test(timeout = 60000)
+	public void testCancelingEmptyTopic() throws Exception {
+		runCancelingOnEmptyInputTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testCancelingFullTopic() throws Exception {
+		runCancelingOnFullInputTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testFailOnDeploy() throws Exception {
+		runFailOnDeployTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testInvalidOffset() throws Exception {
+		
+		final int parallelism = 1;
+		
+		// write 20 messages into topic:
+		final String topic = writeSequence("invalidOffsetTopic", 20, parallelism, 1);
+
+		// set invalid offset:
+		CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
+		ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardProps.getProperty("group.id"), topic, 0, 1234);
+		curatorClient.close();
+
+		// 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);
+	}
+
+	// --- source to partition mappings and exactly once ---
+
+	@Test(timeout = 60000)
+	public void testOneToOneSources() throws Exception {
+		runOneToOneExactlyOnceTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testOneSourceMultiplePartitions() throws Exception {
+		runOneSourceMultiplePartitionsExactlyOnceTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testMultipleSourcesOnePartition() throws Exception {
+		runMultipleSourcesOnePartitionExactlyOnceTest();
+	}
+
+	// --- broker failure ---
+
+	@Test(timeout = 60000)
+	public void testBrokerFailure() throws Exception {
+		runBrokerFailureTest();
+	}
+
+	// --- offset committing ---
+
+	@Test(timeout = 60000)
+	public void testCommitOffsetsToZookeeper() throws Exception {
+		runCommitOffsetsToKafka();
+	}
+
+	@Test(timeout = 60000)
+	public void testStartFromZookeeperCommitOffsets() throws Exception {
+		runStartFromKafkaCommitOffsets();
+	}
+
+	@Test(timeout = 60000)
+	public void testAutoOffsetRetrievalAndCommitToZookeeper() throws Exception {
+		runAutoOffsetRetrievalAndCommitToKafka();
+	}
+
+	@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());
+		}
+	}
+
+	@Test(timeout = 60000)
+	public void testOffsetAutocommitTest() throws Exception {
+		final int parallelism = 3;
+
+		// write a sequence from 0 to 99 to each of the 3 partitions.
+		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
+		// sure that we commit roughly 3-4 times while reading, however
+		// at least once.
+		Properties readProps = new Properties();
+		readProps.putAll(standardProps);
+		readProps.setProperty("auto.commit.interval.ms", "500");
+
+		// read so that the offset can be committed to ZK
+		readSequence(env, readProps, parallelism, topicName, 100, 0);
+
+		// 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);
+		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 != 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);
+	}
+
+	// --- special executions ---
+
+	@Test(timeout = 60000)
+	public void testBigRecordJob() throws Exception {
+		runBigRecordTestTopology();
+	}
+
+	@Test(timeout = 60000)
+	public void testMultipleTopics() throws Exception {
+		runProduceConsumeMultipleTopics();
+	}
+
+	@Test(timeout = 60000)
+	public void testAllDeletes() throws Exception {
+		runAllDeletesTest();
+	}
+
+	@Test(timeout=60000)
+	public void testEndOfStream() throws Exception {
+		runEndOfStreamTest();
+	}
+
+	@Test(timeout = 60000)
+	public void testMetrics() throws Throwable {
+		runMetricsTest();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
new file mode 100644
index 0000000..6d0b140
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import java.util.Properties;
+
+public class Kafka08JsonTableSinkTest extends KafkaTableSinkTestBase {
+
+	@Override
+	protected KafkaTableSink createTableSink(String topic, Properties properties, KafkaPartitioner<Row> partitioner,
+			final FlinkKafkaProducerBase<Row> kafkaProducer) {
+
+		return new Kafka08JsonTableSink(topic, properties, partitioner) {
+			@Override
+			protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties,
+					SerializationSchema<Row> serializationSchema, KafkaPartitioner<Row> partitioner) {
+				return kafkaProducer;
+			}
+		};
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	protected SerializationSchema<Row> getSerializationSchema() {
+		return new JsonRowSerializationSchema(FIELD_NAMES);
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
new file mode 100644
index 0000000..a2d66ac
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import java.util.Properties;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.table.Row;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
+
+public class Kafka08JsonTableSourceTest extends KafkaTableSourceTestBase {
+
+	@Override
+	protected KafkaTableSource createTableSource(String topic, Properties properties, String[] fieldNames, TypeInformation<?>[] typeInfo) {
+		return new Kafka08JsonTableSource(topic, properties, fieldNames, typeInfo);
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	protected Class<DeserializationSchema<Row>> getDeserializationSchema() {
+		return (Class) JsonRowDeserializationSchema.class;
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	protected Class<FlinkKafkaConsumerBase<Row>> getFlinkKafkaConsumer() {
+		return (Class) FlinkKafkaConsumer08.class;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java
new file mode 100644
index 0000000..5c951db
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+
+import org.junit.Test;
+
+@SuppressWarnings("serial")
+public class Kafka08ProducerITCase extends KafkaProducerTestBase {
+
+	@Test
+	public void testCustomPartitioning() {
+		runCustomPartitioningTest();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
new file mode 100644
index 0000000..9520f55
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.Collections;
+import java.util.Properties;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.junit.Test;
+
+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");
+
+			FlinkKafkaConsumer08<String> consumer = new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"), new SimpleStringSchema(), props);
+			consumer.open(new Configuration());
+			fail();
+		}
+		catch (Exception e) {
+			assertTrue(e.getMessage().contains("Unable to retrieve any partitions"));
+		}
+	}
+
+	@Test
+	public void testAllBoostrapServerHostsAreInvalid() {
+		try {
+			String zookeeperConnect = "localhost:56794";
+			String bootstrapServers = "indexistentHost:11111";
+			String groupId = "non-existent-group";
+			Properties props = createKafkaProps(zookeeperConnect, bootstrapServers, groupId);
+			FlinkKafkaConsumer08<String> consumer = new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"),
+					new SimpleStringSchema(), props);
+			consumer.open(new Configuration());
+			fail();
+		} catch (Exception e) {
+			assertTrue("Exception should be thrown containing 'all bootstrap servers invalid' message!",
+					e.getMessage().contains("All the servers provided in: '" + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG
+							+ "' config are invalid"));
+		}
+	}
+
+	@Test
+	public void testAtLeastOneBootstrapServerHostIsValid() {
+		try {
+			String zookeeperConnect = "localhost:56794";
+			// we declare one valid boostrap server, namely the one with
+			// 'localhost'
+			String bootstrapServers = "indexistentHost:11111, localhost:22222";
+			String groupId = "non-existent-group";
+			Properties props = createKafkaProps(zookeeperConnect, bootstrapServers, groupId);
+			FlinkKafkaConsumer08<String> consumer = new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"),
+					new SimpleStringSchema(), props);
+			consumer.open(new Configuration());
+			fail();
+		} catch (Exception e) {
+			// test is not failing because we have one valid boostrap server
+			assertTrue("The cause of the exception should not be 'all boostrap server are invalid'!",
+					!e.getMessage().contains("All the hosts provided in: " + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG
+							+ " config are invalid"));
+		}
+	}
+	
+	private Properties createKafkaProps(String zookeeperConnect, String bootstrapServers, String groupId) {
+		Properties props = new Properties();
+		props.setProperty("zookeeper.connect", zookeeperConnect);
+		props.setProperty("bootstrap.servers", bootstrapServers);
+		props.setProperty("group.id", groupId);
+		return props;
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
new file mode 100644
index 0000000..91fc286
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.operators.StreamSink;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
+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.Collections;
+import java.util.concurrent.Future;
+
+
+import static org.mockito.Mockito.*;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+import static org.junit.Assert.*;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(FlinkKafkaProducerBase.class)
+public class KafkaProducerTest extends TestLogger {
+	
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testPropagateExceptions() {
+		try {
+			// mock kafka producer
+			KafkaProducer<?, ?> kafkaProducerMock = mock(KafkaProducer.class);
+			
+			// partition setup
+			when(kafkaProducerMock.partitionsFor(anyString())).thenReturn(
+				// returning a unmodifiable list to mimic KafkaProducer#partitionsFor() behaviour
+				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)))
+				.thenAnswer(new Answer<Future<RecordMetadata>>() {
+					@Override
+					public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
+						Callback callback = (Callback) invocation.getArguments()[1];
+						callback.onCompletion(null, new Exception("Test error"));
+						return null;
+					}
+				});
+			
+			// make sure the FlinkKafkaProducer instantiates our mock producer
+			whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock);
+			
+			// (1) producer that propagates errors
+
+			FlinkKafkaProducer08<String> producerPropagating = new FlinkKafkaProducer08<>(
+					"mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), null);
+
+			OneInputStreamOperatorTestHarness<String, Object> testHarness =
+					new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producerPropagating));
+
+			testHarness.open();
+
+			try {
+				testHarness.processElement(new StreamRecord<>("value"));
+				testHarness.processElement(new StreamRecord<>("value"));
+				fail("This should fail with an exception");
+			}
+			catch (Exception e) {
+				assertNotNull(e.getCause());
+				assertNotNull(e.getCause().getMessage());
+				assertTrue(e.getCause().getMessage().contains("Test error"));
+			}
+
+			testHarness.close();
+
+			// (2) producer that only logs errors
+
+			FlinkKafkaProducer08<String> producerLogging = new FlinkKafkaProducer08<>(
+					"mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), null);
+			producerLogging.setLogFailuresOnly(true);
+
+			testHarness = new OneInputStreamOperatorTestHarness<>(new StreamSink(producerLogging));
+
+			testHarness.open();
+
+			testHarness.processElement(new StreamRecord<>("value"));
+			testHarness.processElement(new StreamRecord<>("value"));
+
+			testHarness.close();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java
new file mode 100644
index 0000000..c28799c
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.junit.Test;
+
+@SuppressWarnings("serial")
+public class KafkaShortRetention08ITCase extends KafkaShortRetentionTestBase {
+
+	@Test(timeout=60000)
+	public void testAutoOffsetReset() throws Exception {
+		runAutoOffsetResetTest();
+	}
+
+	@Test(timeout=60000)
+	public void testAutoOffsetResetNone() throws Exception {
+		runFailOnAutoOffsetResetNoneEager();
+	}
+}


[17/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java b/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java
deleted file mode 100644
index 650931f..0000000
--- a/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.elasticsearch2;
-
-import org.elasticsearch.action.ActionRequest;
-import org.elasticsearch.action.bulk.BulkProcessor;
-
-public class BulkProcessorIndexer implements RequestIndexer {
-	private final BulkProcessor bulkProcessor;
-
-	public BulkProcessorIndexer(BulkProcessor bulkProcessor) {
-		this.bulkProcessor = bulkProcessor;
-	}
-
-	@Override
-	public void add(ActionRequest... actionRequests) {
-		for (ActionRequest actionRequest : actionRequests) {
-			this.bulkProcessor.add(actionRequest);
-		}
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java b/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java
deleted file mode 100644
index 55ba720..0000000
--- a/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.elasticsearch2;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.RuntimeContext;
-
-import java.io.Serializable;
-
-/**
- * Method that creates an {@link org.elasticsearch.action.ActionRequest} from an element in a Stream.
- *
- * <p>
- * This is used by {@link ElasticsearchSink} to prepare elements for sending them to Elasticsearch.
- *
- * <p>
- * Example:
- *
- * <pre>{@code
- *					private static class TestElasticSearchSinkFunction implements
- *						ElasticsearchSinkFunction<Tuple2<Integer, String>> {
- *
- *					public IndexRequest createIndexRequest(Tuple2<Integer, String> element) {
- *						Map<String, Object> json = new HashMap<>();
- *						json.put("data", element.f1);
- *
- *						return Requests.indexRequest()
- *							.index("my-index")
- *							.type("my-type")
- *							.id(element.f0.toString())
- *							.source(json);
- *						}
- *
- *				public void process(Tuple2<Integer, String> element, RuntimeContext ctx, RequestIndexer indexer) {
- *					indexer.add(createIndexRequest(element));
- *				}
- *		}
- *
- * }</pre>
- *
- * @param <T> The type of the element handled by this {@code ElasticsearchSinkFunction}
- */
-public interface ElasticsearchSinkFunction<T> extends Serializable, Function {
-	void process(T element, RuntimeContext ctx, RequestIndexer indexer);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java b/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java
deleted file mode 100644
index 144a87b..0000000
--- a/flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.elasticsearch2;
-
-import org.elasticsearch.action.ActionRequest;
-
-import java.io.Serializable;
-
-public interface RequestIndexer extends Serializable {
-	void add(ActionRequest... actionRequests);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java b/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
deleted file mode 100644
index bc9bedc..0000000
--- a/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.elasticsearch2;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.elasticsearch.action.get.GetRequest;
-import org.elasticsearch.action.get.GetResponse;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.client.Requests;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.node.Node;
-import org.elasticsearch.node.NodeBuilder;
-import org.junit.Assert;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase {
-
-	private static final int NUM_ELEMENTS = 20;
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Test
-	public void testTransportClient() throws Exception {
-
-		File dataDir = tempFolder.newFolder();
-
-		Node node = NodeBuilder.nodeBuilder()
-				.settings(Settings.settingsBuilder()
-						.put("path.home", dataDir.getParent())
-						.put("http.enabled", false)
-						.put("path.data", dataDir.getAbsolutePath()))
-				// set a custom cluster name to verify that user config works correctly
-				.clusterName("my-transport-client-cluster")
-				.node();
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
-
-		Map<String, String> config = new HashMap<>();
-		// This instructs the sink to emit after every element, otherwise they would be buffered
-		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-		config.put("cluster.name", "my-transport-client-cluster");
-
-		// Can't use {@link TransportAddress} as its not Serializable in Elasticsearch 2.x
-		List<InetSocketAddress> transports = new ArrayList<>();
-		transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
-
-		source.addSink(new ElasticsearchSink<>(config, transports, new TestElasticsearchSinkFunction()));
-
-		env.execute("Elasticsearch TransportClient Test");
-
-		// verify the results
-		Client client = node.client();
-		for (int i = 0; i < NUM_ELEMENTS; i++) {
-			GetResponse response = client.get(new GetRequest("my-index",
-					"my-type", Integer.toString(i))).actionGet();
-			Assert.assertEquals("message #" + i, response.getSource().get("data"));
-		}
-
-		node.close();
-	}
-
- @Test(expected = IllegalArgumentException.class)
- public void testNullTransportClient() throws Exception {
-
-	File dataDir = tempFolder.newFolder();
-
-	Node node = NodeBuilder.nodeBuilder()
-		.settings(Settings.settingsBuilder()
-			.put("path.home", dataDir.getParent())
-			.put("http.enabled", false)
-			.put("path.data", dataDir.getAbsolutePath()))
-		// set a custom cluster name to verify that user config works correctly
-		.clusterName("my-transport-client-cluster")
-		.node();
-
-	final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-	DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
-
-	Map<String, String> config = new HashMap<>();
-	// This instructs the sink to emit after every element, otherwise they would be buffered
-	config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-	config.put("cluster.name", "my-transport-client-cluster");
-
-	source.addSink(new ElasticsearchSink<>(config, null, new TestElasticsearchSinkFunction()));
-
-	env.execute("Elasticsearch TransportClient Test");
-
-	// verify the results
-	Client client = node.client();
-	for (int i = 0; i < NUM_ELEMENTS; i++) {
-	 GetResponse response = client.get(new GetRequest("my-index",
-		 "my-type", Integer.toString(i))).actionGet();
-	 Assert.assertEquals("message #" + i, response.getSource().get("data"));
-	}
-
-	node.close();
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testEmptyTransportClient() throws Exception {
-
-	File dataDir = tempFolder.newFolder();
-
-	Node node = NodeBuilder.nodeBuilder()
-		.settings(Settings.settingsBuilder()
-			.put("path.home", dataDir.getParent())
-			.put("http.enabled", false)
-			.put("path.data", dataDir.getAbsolutePath()))
-		// set a custom cluster name to verify that user config works correctly
-		.clusterName("my-transport-client-cluster")
-		.node();
-
-	final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-	DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
-
-	Map<String, String> config = new HashMap<>();
-	// This instructs the sink to emit after every element, otherwise they would be buffered
-	config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-	config.put("cluster.name", "my-transport-client-cluster");
-
-	source.addSink(new ElasticsearchSink<>(config, new ArrayList<InetSocketAddress>(), new TestElasticsearchSinkFunction()));
-
-	env.execute("Elasticsearch TransportClient Test");
-
-	// verify the results
-	Client client = node.client();
-	for (int i = 0; i < NUM_ELEMENTS; i++) {
-	 GetResponse response = client.get(new GetRequest("my-index",
-		 "my-type", Integer.toString(i))).actionGet();
-	 Assert.assertEquals("message #" + i, response.getSource().get("data"));
-	}
-
-	node.close();
- }
-
-	@Test(expected = JobExecutionException.class)
-	public void testTransportClientFails() throws Exception{
-		// this checks whether the TransportClient fails early when there is no cluster to
-		// connect to. There isn't a similar test for the Node Client version since that
-		// one will block and wait for a cluster to come online
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
-
-		Map<String, String> config = new HashMap<>();
-		// This instructs the sink to emit after every element, otherwise they would be buffered
-		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-		config.put("cluster.name", "my-node-client-cluster");
-
-		List<InetSocketAddress> transports = new ArrayList<>();
-		transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
-
-		source.addSink(new ElasticsearchSink<>(config, transports, new TestElasticsearchSinkFunction()));
-
-		env.execute("Elasticsearch Node Client Test");
-	}
-
-	private static class TestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		private volatile boolean running = true;
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
-			for (int i = 0; i < NUM_ELEMENTS && running; i++) {
-				ctx.collect(Tuple2.of(i, "message #" + i));
-			}
-		}
-
-		@Override
-		public void cancel() {
-			running = false;
-		}
-	}
-
-	private static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		public IndexRequest createIndexRequest(Tuple2<Integer, String> element) {
-			Map<String, Object> json = new HashMap<>();
-			json.put("data", element.f1);
-
-			return Requests.indexRequest()
-					.index("my-index")
-					.type("my-type")
-					.id(element.f0.toString())
-					.source(json);
-		}
-
-		@Override
-		public void process(Tuple2<Integer, String> element, RuntimeContext ctx, RequestIndexer indexer) {
-			indexer.add(createIndexRequest(element));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java b/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java
deleted file mode 100644
index 05760e8..0000000
--- a/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.elasticsearch2.examples;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink;
-import org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSinkFunction;
-import org.apache.flink.streaming.connectors.elasticsearch2.RequestIndexer;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.client.Requests;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * This example shows how to use the Elasticsearch Sink. Before running it you must ensure that
- * you have a cluster named "elasticsearch" running or change the name of cluster in the config map.
- */
-public class ElasticsearchExample {
-
-	public static void main(String[] args) throws Exception {
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		SingleOutputStreamOperator<String> source =
-				env.generateSequence(0, 20).map(new MapFunction<Long, String>() {
-					/**
-					 * The mapping method. Takes an element from the input data set and transforms
-					 * it into exactly one element.
-					 *
-					 * @param value The input value.
-					 * @return The transformed value
-					 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
-					 *                   to fail and may trigger recovery.
-					 */
-					@Override
-					public String map(Long value) throws Exception {
-						return "message #" + value;
-					}
-				});
-
-		Map<String, String> config = new HashMap<>();
-		// This instructs the sink to emit after every element, otherwise they would be buffered
-		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-
-		List<InetSocketAddress> transports = new ArrayList<>();
-		transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
-
-		source.addSink(new ElasticsearchSink<>(config, transports, new ElasticsearchSinkFunction<String>(){
-			@Override
-			public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
-				indexer.add(createIndexRequest(element));
-			}
-		}));
-
-		env.execute("Elasticsearch Example");
-	}
-
-	private static IndexRequest createIndexRequest(String element) {
-		Map<String, Object> json = new HashMap<>();
-		json.put("data", element);
-
-		return Requests.indexRequest()
-				.index("my-index")
-				.type("my-type")
-				.id(element)
-				.source(json);
-	}
-}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/pom.xml b/flink-streaming-connectors/flink-connector-filesystem/pom.xml
deleted file mode 100644
index 20c48c6..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/pom.xml
+++ /dev/null
@@ -1,163 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-filesystem_2.10</artifactId>
-	<name>flink-connector-filesystem</name>
-
-	<packaging>jar</packaging>
-
-	<!--
-		This is a Hadoop2 only flink module.
-	-->
-
-	<dependencies>
-
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop2</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils-junit</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-		
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-hdfs</artifactId>
-			<scope>test</scope>
-			<type>test-jar</type>
-			<version>${hadoop.version}</version><!--$NO-MVN-MAN-VER$-->
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>test</scope>
-			<type>test-jar</type>
-			<version>${hadoop.version}</version><!--$NO-MVN-MAN-VER$-->
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-minikdc</artifactId>
-			<version>${minikdc.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-
-			<!--
-				https://issues.apache.org/jira/browse/DIRSHARED-134
-				Required to pull the Mini-KDC transitive dependency
-			-->
-			<plugin>
-				<groupId>org.apache.felix</groupId>
-				<artifactId>maven-bundle-plugin</artifactId>
-				<version>3.0.1</version>
-				<inherited>true</inherited>
-				<extensions>true</extensions>
-			</plugin>
-
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<!--
-					Enforce single threaded execution to avoid port conflicts when running
-					secure mini DFS cluster
-					-->
-					<forkCount>1</forkCount>
-					<reuseForks>false</reuseForks>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
deleted file mode 100644
index 3e3c86b..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
+++ /dev/null
@@ -1,309 +0,0 @@
-package org.apache.flink.streaming.connectors.fs;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.file.CodecFactory;
-import org.apache.avro.file.DataFileConstants;
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumWriter;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-
-/**
-* Implementation of AvroKeyValue writer that can be used in Sink.
-* Each entry would be wrapped in GenericRecord with key/value fields(same as in m/r lib)
-<pre>
-Usage:
-{@code
-		BucketingSink<Tuple2<Long, Long>> sink = new BucketingSink<Tuple2<Long, Long>>("/tmp/path");
-		sink.setBucketer(new DateTimeBucketer<Tuple2<Long, Long>>("yyyy-MM-dd/HH/mm/"));
-		sink.setPendingSuffix(".avro");
-		Map<String,String> properties = new HashMap<>();
-		Schema longSchema = Schema.create(Type.LONG);
-		String keySchema = longSchema.toString();
-		String valueSchema = longSchema.toString();
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_KEY_SCHEMA, keySchema);
-		properties.put(AvroKeyValueSinkWriter.CONF_OUTPUT_VALUE_SCHEMA, valueSchema);
-		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS, Boolean.toString(true));
-		properties.put(AvroKeyValueSinkWriter.CONF_COMPRESS_CODEC, DataFileConstants.SNAPPY_CODEC);
-		
-		sink.setWriter(new AvroSinkWriter<Long, Long>(properties));
-		sink.setBatchSize(1024 * 1024 * 64); // this is 64 MB,
-}
-</pre>
-*/
-public class AvroKeyValueSinkWriter<K, V> extends StreamWriterBase<Tuple2<K, V>>  implements Writer<Tuple2<K, V>>, InputTypeConfigurable {
-	private static final long serialVersionUID = 1L;
-	public static final String CONF_OUTPUT_KEY_SCHEMA = "avro.schema.output.key";
-	public static final String CONF_OUTPUT_VALUE_SCHEMA = "avro.schema.output.value";
-	public static final String CONF_COMPRESS = FileOutputFormat.COMPRESS;
-	public static final String CONF_COMPRESS_CODEC = FileOutputFormat.COMPRESS_CODEC;
-	public static final String CONF_DEFLATE_LEVEL = "avro.deflate.level";
-	public static final String CONF_XZ_LEVEL = "avro.xz.level";
-
-	private transient AvroKeyValueWriter<K, V> keyValueWriter;
-
-	private final Map<String, String> properties;
-
-	/**
-	 * C'tor for the writer
-	 * <p>
-	 * You can provide different properties that will be used to configure avro key-value writer as simple properties map(see example above)
-	 * @param properties
-	 */
-	@SuppressWarnings("deprecation")
-	public AvroKeyValueSinkWriter(Map<String, String> properties) {
-		this.properties = properties;
-		
-		String keySchemaString = properties.get(CONF_OUTPUT_KEY_SCHEMA);
-		if (keySchemaString == null) {
-			throw new IllegalStateException("No key schema provided, set '" + CONF_OUTPUT_KEY_SCHEMA + "' property");
-		}
-		Schema.parse(keySchemaString);//verifying that schema valid
-		
-		String valueSchemaString = properties.get(CONF_OUTPUT_VALUE_SCHEMA);
-		if (valueSchemaString == null) {
-			throw new IllegalStateException("No value schema provided, set '" + CONF_OUTPUT_VALUE_SCHEMA + "' property");
-		}
-		Schema.parse(valueSchemaString);//verifying that schema valid
-	}
-
-	private boolean getBoolean(Map<String,String> conf, String key, boolean def) {
-		String value = conf.get(key);
-		if (value == null) {
-			return def;
-		}
-		return Boolean.parseBoolean(value);
-	}
-	
-	private int getInt(Map<String,String> conf, String key, int def) {
-		String value = conf.get(key);
-		if (value == null) {
-			return def;
-		}
-		return Integer.parseInt(value);
-	}
-
-	//this derived from AvroOutputFormatBase.getCompressionCodec(..)
-	private CodecFactory getCompressionCodec(Map<String,String> conf) {
-		if (getBoolean(conf, CONF_COMPRESS, false)) {
-			int deflateLevel = getInt(conf, CONF_DEFLATE_LEVEL, CodecFactory.DEFAULT_DEFLATE_LEVEL);
-			int xzLevel = getInt(conf, CONF_XZ_LEVEL, CodecFactory.DEFAULT_XZ_LEVEL);
-
-			String outputCodec = conf.get(CONF_COMPRESS_CODEC);
-
-			if (DataFileConstants.DEFLATE_CODEC.equals(outputCodec)) {
-				return CodecFactory.deflateCodec(deflateLevel);
-			} else if (DataFileConstants.XZ_CODEC.equals(outputCodec)) {
-				return CodecFactory.xzCodec(xzLevel);
-			} else {
-				return CodecFactory.fromString(outputCodec);
-			}
-		}
-		return CodecFactory.nullCodec();
-	}
-
-	@Override
-	@SuppressWarnings("deprecation")
-	public void open(FileSystem fs, Path path) throws IOException {
-		super.open(fs, path);
-
-		CodecFactory compressionCodec = getCompressionCodec(properties);
-		Schema keySchema = Schema.parse(properties.get(CONF_OUTPUT_KEY_SCHEMA));
-		Schema valueSchema = Schema.parse(properties.get(CONF_OUTPUT_VALUE_SCHEMA));
-		keyValueWriter = new AvroKeyValueWriter<K, V>(keySchema, valueSchema, compressionCodec, getStream());
-	}
-
-	@Override
-	public void close() throws IOException {
-		super.close();//the order is important since super.close flushes inside
-		if (keyValueWriter != null) {
-			keyValueWriter.close();
-		}
-	}
-	
-	@Override
-	public long flush() throws IOException {
-		if (keyValueWriter != null) {
-			keyValueWriter.sync();
-		}
-		return super.flush();
-	}
-
-	@Override
-	public void write(Tuple2<K, V> element) throws IOException {
-		getStream(); // Throws if the stream is not open
-		keyValueWriter.write(element.f0, element.f1);
-	}
-
-	@Override
-	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
-		if (!type.isTupleType()) {
-			throw new IllegalArgumentException("Input TypeInformation is not a tuple type.");
-		}
-
-		TupleTypeInfoBase<?> tupleType = (TupleTypeInfoBase<?>) type;
-
-		if (tupleType.getArity() != 2) {
-			throw new IllegalArgumentException("Input TypeInformation must be a Tuple2 type.");
-		}
-	}
-
-	@Override
-	public Writer<Tuple2<K, V>> duplicate() {
-		return new AvroKeyValueSinkWriter<K, V>(properties);
-	}
-	
-	// taken from m/r avro lib to remove dependency on it
-	private static final class AvroKeyValueWriter<K, V> {
-		/** A writer for the Avro container file. */
-		private final DataFileWriter<GenericRecord> mAvroFileWriter;
-
-		/**
-		 * The writer schema for the generic record entries of the Avro
-		 * container file.
-		 */
-		private final Schema mKeyValuePairSchema;
-
-		/**
-		 * A reusable Avro generic record for writing key/value pairs to the
-		 * file.
-		 */
-		private final AvroKeyValue<Object, Object> mOutputRecord;
-
-		AvroKeyValueWriter(Schema keySchema, Schema valueSchema,
-				CodecFactory compressionCodec, OutputStream outputStream,
-				int syncInterval) throws IOException {
-			// Create the generic record schema for the key/value pair.
-			mKeyValuePairSchema = AvroKeyValue
-					.getSchema(keySchema, valueSchema);
-
-			// Create an Avro container file and a writer to it.
-			DatumWriter<GenericRecord> genericDatumWriter = new GenericDatumWriter<GenericRecord>(
-					mKeyValuePairSchema);
-			mAvroFileWriter = new DataFileWriter<GenericRecord>(
-					genericDatumWriter);
-			mAvroFileWriter.setCodec(compressionCodec);
-			mAvroFileWriter.setSyncInterval(syncInterval);
-			mAvroFileWriter.create(mKeyValuePairSchema, outputStream);
-
-			// Create a reusable output record.
-			mOutputRecord = new AvroKeyValue<Object, Object>(
-					new GenericData.Record(mKeyValuePairSchema));
-		}
-
-		AvroKeyValueWriter(Schema keySchema, Schema valueSchema,
-				CodecFactory compressionCodec, OutputStream outputStream)
-				throws IOException {
-			this(keySchema, valueSchema, compressionCodec, outputStream,
-					DataFileConstants.DEFAULT_SYNC_INTERVAL);
-		}
-
-		void write(K key, V value) throws IOException {
-			mOutputRecord.setKey(key);
-			mOutputRecord.setValue(value);
-			mAvroFileWriter.append(mOutputRecord.get());
-		}
-
-		void close() throws IOException {
-			mAvroFileWriter.close();
-		}
-
-		long sync() throws IOException {
-			return mAvroFileWriter.sync();
-		}
-	}
-
-	// taken from AvroKeyValue avro-mapr lib
-	public static class AvroKeyValue<K, V> {
-		/** The name of the key value pair generic record. */
-		public static final String KEY_VALUE_PAIR_RECORD_NAME = "KeyValuePair";
-
-		/** The namespace of the key value pair generic record. */
-		public static final String KEY_VALUE_PAIR_RECORD_NAMESPACE = "org.apache.avro.mapreduce";
-
-		/** The name of the generic record field containing the key. */
-		public static final String KEY_FIELD = "key";
-
-		/** The name of the generic record field containing the value. */
-		public static final String VALUE_FIELD = "value";
-
-		/** The key/value generic record wrapped by this class. */
-		public final GenericRecord mKeyValueRecord;
-
-		/**
-		 * Wraps a GenericRecord that is a key value pair.
-		 */
-		public AvroKeyValue(GenericRecord keyValueRecord) {
-			mKeyValueRecord = keyValueRecord;
-		}
-
-		public GenericRecord get() {
-			return mKeyValueRecord;
-		}
-
-		public void setKey(K key) {
-			mKeyValueRecord.put(KEY_FIELD, key);
-		}
-
-		public void setValue(V value) {
-			mKeyValueRecord.put(VALUE_FIELD, value);
-		}
-
-		@SuppressWarnings("unchecked")
-		public K getKey() {
-			return (K) mKeyValueRecord.get(KEY_FIELD);
-		}
-
-		@SuppressWarnings("unchecked")
-		public V getValue() {
-			return (V) mKeyValueRecord.get(VALUE_FIELD);
-		}
-
-		/**
-		 * Creates a KeyValuePair generic record schema.
-		 * 
-		 * @return A schema for a generic record with two fields: 'key' and
-		 *         'value'.
-		 */
-		public static Schema getSchema(Schema keySchema, Schema valueSchema) {
-			Schema schema = Schema.createRecord(KEY_VALUE_PAIR_RECORD_NAME,
-					"A key/value pair", KEY_VALUE_PAIR_RECORD_NAMESPACE, false);
-			schema.setFields(Arrays.asList(new Schema.Field(KEY_FIELD,
-					keySchema, "The key", null), new Schema.Field(VALUE_FIELD,
-					valueSchema, "The value", null)));
-			return schema;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java
deleted file mode 100644
index 24ad6ab..0000000
--- a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.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.fs;
-
-import org.apache.hadoop.fs.Path;
-
-import java.io.Serializable;
-
-/**
- * A bucketer is used with a {@link RollingSink}
- * to put emitted elements into rolling files.
- *
- * <p>
- * The {@code RollingSink} has one active bucket that it is writing to at a time. Whenever
- * a new element arrives it will ask the {@code Bucketer} if a new bucket should be started and
- * the old one closed. The {@code Bucketer} can, for example, decide to start new buckets
- * based on system time.
- *
- * @deprecated use {@link org.apache.flink.streaming.connectors.fs.bucketing.Bucketer} instead.
- */
-@Deprecated
-public interface Bucketer extends Serializable {
-
-	/**
-	 * Returns {@code true} when a new bucket should be started.
-	 *
-	 * @param currentBucketPath The bucket {@code Path} that is currently being used.
-	 */
-	boolean shouldStartNewBucket(Path basePath, Path currentBucketPath);
-
-	/**
-	 * Returns the {@link Path} of a new bucket file.
-	 *
-	 * @param basePath The base path containing all the buckets.
-	 *
-	 * @return The complete new {@code Path} of the new bucket. This should include the {@code basePath}
-	 *      and also the {@code subtaskIndex} tp avoid clashes with parallel sinks.
-	 */
-	Path getNextBucketPath(Path basePath);
-}

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

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

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


[25/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java
new file mode 100644
index 0000000..65e6d4e
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.testutils;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import com.amazonaws.services.kinesis.model.Record;
+import com.amazonaws.services.kinesis.model.Shard;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import java.nio.ByteBuffer;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * Factory for different kinds of fake Kinesis behaviours using the {@link KinesisProxyInterface} interface.
+ */
+public class FakeKinesisBehavioursFactory {
+
+	// ------------------------------------------------------------------------
+	//  Behaviours related to shard listing and resharding, used in KinesisDataFetcherTest
+	// ------------------------------------------------------------------------
+
+	public static KinesisProxyInterface noShardsFoundForRequestedStreamsBehaviour() {
+
+		return new KinesisProxyInterface() {
+			@Override
+			public GetShardListResult getShardList(Map<String, String> streamNamesWithLastSeenShardIds) {
+				return new GetShardListResult(); // not setting any retrieved shards for result
+			}
+
+			@Override
+			public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) {
+				return null;
+			}
+
+			@Override
+			public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
+				return null;
+			}
+		};
+
+	}
+
+	public static KinesisProxyInterface nonReshardedStreamsBehaviour(Map<String,Integer> streamsToShardCount) {
+		return new NonReshardedStreamsKinesis(streamsToShardCount);
+
+	}
+
+	// ------------------------------------------------------------------------
+	//  Behaviours related to fetching records, used mainly in ShardConsumerTest
+	// ------------------------------------------------------------------------
+
+	public static KinesisProxyInterface totalNumOfRecordsAfterNumOfGetRecordsCalls(final int numOfRecords, final int numOfGetRecordsCalls) {
+		return new SingleShardEmittingFixNumOfRecordsKinesis(numOfRecords, numOfGetRecordsCalls);
+	}
+	
+	public static KinesisProxyInterface totalNumOfRecordsAfterNumOfGetRecordsCallsWithUnexpectedExpiredIterator(
+		final int numOfRecords, final int numOfGetRecordsCall, final int orderOfCallToExpire) {
+		return new SingleShardEmittingFixNumOfRecordsWithExpiredIteratorKinesis(
+			numOfRecords, numOfGetRecordsCall, orderOfCallToExpire);
+	}
+
+	public static class SingleShardEmittingFixNumOfRecordsWithExpiredIteratorKinesis extends SingleShardEmittingFixNumOfRecordsKinesis {
+
+		private boolean expiredOnceAlready = false;
+		private boolean expiredIteratorRefreshed = false;
+		private final int orderOfCallToExpire;
+
+		public SingleShardEmittingFixNumOfRecordsWithExpiredIteratorKinesis(final int numOfRecords,
+																			final int numOfGetRecordsCalls,
+																			final int orderOfCallToExpire) {
+			super(numOfRecords, numOfGetRecordsCalls);
+			checkArgument(orderOfCallToExpire <= numOfGetRecordsCalls,
+				"can not test unexpected expired iterator if orderOfCallToExpire is larger than numOfGetRecordsCalls");
+			this.orderOfCallToExpire = orderOfCallToExpire;
+		}
+
+		@Override
+		public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
+			if ((Integer.valueOf(shardIterator) == orderOfCallToExpire-1) && !expiredOnceAlready) {
+				// we fake only once the expired iterator exception at the specified get records attempt order
+				expiredOnceAlready = true;
+				throw new ExpiredIteratorException("Artificial expired shard iterator");
+			} else if (expiredOnceAlready && !expiredIteratorRefreshed) {
+				// if we've thrown the expired iterator exception already, but the iterator was not refreshed,
+				// throw a hard exception to the test that is testing this Kinesis behaviour
+				throw new RuntimeException("expired shard iterator was not refreshed on the next getRecords() call");
+			} else {
+				// assuming that the maxRecordsToGet is always large enough
+				return new GetRecordsResult()
+					.withRecords(shardItrToRecordBatch.get(shardIterator))
+					.withNextShardIterator(
+						(Integer.valueOf(shardIterator) == totalNumOfGetRecordsCalls - 1)
+							? null : String.valueOf(Integer.valueOf(shardIterator) + 1)); // last next shard iterator is null
+			}
+		}
+
+		@Override
+		public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) {
+			if (!expiredOnceAlready) {
+				// for the first call, just return the iterator of the first batch of records
+				return "0";
+			} else {
+				// fake the iterator refresh when this is called again after getRecords throws expired iterator
+				// exception on the orderOfCallToExpire attempt
+				expiredIteratorRefreshed = true;
+				return String.valueOf(orderOfCallToExpire-1);
+			}
+		}
+	}
+
+	private static class SingleShardEmittingFixNumOfRecordsKinesis implements KinesisProxyInterface {
+
+		protected final int totalNumOfGetRecordsCalls;
+
+		protected final int totalNumOfRecords;
+
+		protected final Map<String,List<Record>> shardItrToRecordBatch;
+
+		public SingleShardEmittingFixNumOfRecordsKinesis(final int numOfRecords, final int numOfGetRecordsCalls) {
+			this.totalNumOfRecords = numOfRecords;
+			this.totalNumOfGetRecordsCalls = numOfGetRecordsCalls;
+
+			// initialize the record batches that we will be fetched
+			this.shardItrToRecordBatch = new HashMap<>();
+
+			int numOfAlreadyPartitionedRecords = 0;
+			int numOfRecordsPerBatch = numOfRecords/numOfGetRecordsCalls + 1;
+			for (int batch=0; batch<totalNumOfGetRecordsCalls; batch++) {
+				if (batch != totalNumOfGetRecordsCalls-1) {
+					shardItrToRecordBatch.put(
+						String.valueOf(batch),
+						createRecordBatchWithRange(
+							numOfAlreadyPartitionedRecords,
+							numOfAlreadyPartitionedRecords + numOfRecordsPerBatch));
+					numOfAlreadyPartitionedRecords += numOfRecordsPerBatch;
+				} else {
+					shardItrToRecordBatch.put(
+						String.valueOf(batch),
+						createRecordBatchWithRange(
+							numOfAlreadyPartitionedRecords,
+							totalNumOfRecords));
+				}
+			}
+		}
+
+		@Override
+		public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
+			// assuming that the maxRecordsToGet is always large enough
+			return new GetRecordsResult()
+				.withRecords(shardItrToRecordBatch.get(shardIterator))
+				.withNextShardIterator(
+					(Integer.valueOf(shardIterator) == totalNumOfGetRecordsCalls-1)
+						? null : String.valueOf(Integer.valueOf(shardIterator)+1)); // last next shard iterator is null
+		}
+
+		@Override
+		public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) {
+			// this will be called only one time per ShardConsumer;
+			// so, simply return the iterator of the first batch of records
+			return "0";
+		}
+
+		@Override
+		public GetShardListResult getShardList(Map<String, String> streamNamesWithLastSeenShardIds) {
+			return null;
+		}
+
+		public static List<Record> createRecordBatchWithRange(int min, int max) {
+			List<Record> batch = new LinkedList<>();
+			for (int i = min; i < max; i++) {
+				batch.add(
+					new Record()
+						.withData(ByteBuffer.wrap(String.valueOf(i).getBytes()))
+						.withPartitionKey(UUID.randomUUID().toString())
+						.withApproximateArrivalTimestamp(new Date(System.currentTimeMillis()))
+						.withSequenceNumber(String.valueOf(i)));
+			}
+			return batch;
+		}
+
+	}
+
+	private static class NonReshardedStreamsKinesis implements KinesisProxyInterface {
+
+		private Map<String, List<KinesisStreamShard>> streamsWithListOfShards = new HashMap<>();
+
+		public NonReshardedStreamsKinesis(Map<String,Integer> streamsToShardCount) {
+			for (Map.Entry<String,Integer> streamToShardCount : streamsToShardCount.entrySet()) {
+				String streamName = streamToShardCount.getKey();
+				int shardCount = streamToShardCount.getValue();
+
+				if (shardCount == 0) {
+					// don't do anything
+				} else {
+					List<KinesisStreamShard> shardsOfStream = new ArrayList<>(shardCount);
+					for (int i=0; i < shardCount; i++) {
+						shardsOfStream.add(
+							new KinesisStreamShard(
+								streamName,
+								new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(i))));
+					}
+					streamsWithListOfShards.put(streamName, shardsOfStream);
+				}
+			}
+		}
+
+		@Override
+		public GetShardListResult getShardList(Map<String, String> streamNamesWithLastSeenShardIds) {
+			GetShardListResult result = new GetShardListResult();
+			for (Map.Entry<String, List<KinesisStreamShard>> streamsWithShards : streamsWithListOfShards.entrySet()) {
+				String streamName = streamsWithShards.getKey();
+				for (KinesisStreamShard shard : streamsWithShards.getValue()) {
+					if (streamNamesWithLastSeenShardIds.get(streamName) == null) {
+						result.addRetrievedShardToStream(streamName, shard);
+					} else {
+						if (KinesisStreamShard.compareShardIds(
+							shard.getShard().getShardId(), streamNamesWithLastSeenShardIds.get(streamName)) > 0) {
+							result.addRetrievedShardToStream(streamName, shard);
+						}
+					}
+				}
+			}
+			return result;
+		}
+
+		@Override
+		public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) {
+			return null;
+		}
+
+		@Override
+		public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
+			return null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisEventsGeneratorProducerThread.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisEventsGeneratorProducerThread.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisEventsGeneratorProducerThread.java
new file mode 100644
index 0000000..fdfdfe1
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisEventsGeneratorProducerThread.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.testutils;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A thread that runs a topology with a manual data generator as source, and the FlinkKinesisProducer as sink.
+ */
+public class KinesisEventsGeneratorProducerThread {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KinesisEventsGeneratorProducerThread.class);
+
+	public static Thread create(final int totalEventCount,
+								final int parallelism,
+								final String awsAccessKey,
+								final String awsSecretKey,
+								final String awsRegion,
+								final String kinesisStreamName,
+								final AtomicReference<Throwable> errorHandler,
+								final int flinkPort,
+								final Configuration flinkConfig) {
+		Runnable kinesisEventsGeneratorProducer = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort, flinkConfig);
+					see.setParallelism(parallelism);
+
+					// start data generator
+					DataStream<String> simpleStringStream = see.addSource(new KinesisEventsGeneratorProducerThread.EventsGenerator(totalEventCount)).setParallelism(1);
+
+					Properties producerProps = new Properties();
+					producerProps.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, awsAccessKey);
+					producerProps.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, awsSecretKey);
+					producerProps.setProperty(AWSConfigConstants.AWS_REGION, awsRegion);
+
+					FlinkKinesisProducer<String> kinesis = new FlinkKinesisProducer<>(new SimpleStringSchema(),
+						producerProps);
+
+					kinesis.setFailOnError(true);
+					kinesis.setDefaultStream(kinesisStreamName);
+					kinesis.setDefaultPartition("0");
+					simpleStringStream.addSink(kinesis);
+
+					LOG.info("Starting producing topology");
+					see.execute("Producing topology");
+					LOG.info("Producing topo finished");
+				} catch (Exception e) {
+					LOG.warn("Error while running producing topology", e);
+					errorHandler.set(e);
+				}
+			}
+		};
+
+		return new Thread(kinesisEventsGeneratorProducer);
+	}
+
+	private static class EventsGenerator implements SourceFunction<String> {
+
+		private static final Logger LOG = LoggerFactory.getLogger(EventsGenerator.class);
+
+		private boolean running = true;
+		private final long limit;
+
+		public EventsGenerator(long limit) {
+			this.limit = limit;
+		}
+
+		@Override
+		public void run(SourceContext<String> ctx) throws Exception {
+			long seq = 0;
+			while(running) {
+				Thread.sleep(10);
+				String evt = (seq++) + "-" + RandomStringUtils.randomAlphabetic(12);
+				ctx.collect(evt);
+				LOG.info("Emitting event {}", evt);
+				if(seq >= limit) {
+					break;
+				}
+			}
+			ctx.close();
+			LOG.info("Stopping events generator");
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java
new file mode 100644
index 0000000..c8dd347
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.testutils;
+
+public class KinesisShardIdGenerator {
+	// Kinesis shards ids are in the form of: ^shardId-\d{12}
+	public static String generateFromShardOrder(int order) {
+		return String.format("shardId-%012d", order);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java
new file mode 100644
index 0000000..80ad06c
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.testutils;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Properties;
+
+public class TestableFlinkKinesisConsumer extends FlinkKinesisConsumer<String> {
+
+	private final RuntimeContext mockedRuntimeCtx;
+
+	public TestableFlinkKinesisConsumer(String fakeStream,
+										Properties fakeConfiguration,
+										final int totalNumOfConsumerSubtasks,
+										final int indexOfThisConsumerSubtask) {
+		super(fakeStream, new SimpleStringSchema(), fakeConfiguration);
+
+		this.mockedRuntimeCtx = Mockito.mock(RuntimeContext.class);
+
+		Mockito.when(mockedRuntimeCtx.getNumberOfParallelSubtasks()).thenAnswer(new Answer<Integer>() {
+			@Override
+			public Integer answer(InvocationOnMock invocationOnMock) throws Throwable {
+				return totalNumOfConsumerSubtasks;
+			}
+		});
+
+		Mockito.when(mockedRuntimeCtx.getIndexOfThisSubtask()).thenAnswer(new Answer<Integer>() {
+			@Override
+			public Integer answer(InvocationOnMock invocationOnMock) throws Throwable {
+				return indexOfThisConsumerSubtask;
+			}
+		});
+	}
+
+	@Override
+	public RuntimeContext getRuntimeContext() {
+		return this.mockedRuntimeCtx;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableKinesisDataFetcher.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableKinesisDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableKinesisDataFetcher.java
new file mode 100644
index 0000000..57886fe
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableKinesisDataFetcher.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.testutils;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class TestableKinesisDataFetcher extends KinesisDataFetcher<String> {
+
+	private static final Object fakeCheckpointLock = new Object();
+
+	private long numElementsCollected;
+
+	public TestableKinesisDataFetcher(List<String> fakeStreams,
+									  Properties fakeConfiguration,
+									  int fakeTotalCountOfSubtasks,
+									  int fakeTndexOfThisSubtask,
+									  AtomicReference<Throwable> thrownErrorUnderTest,
+									  LinkedList<KinesisStreamShardState> subscribedShardsStateUnderTest,
+									  HashMap<String, String> subscribedStreamsToLastDiscoveredShardIdsStateUnderTest,
+									  KinesisProxyInterface fakeKinesis) {
+		super(fakeStreams,
+			getMockedSourceContext(),
+			fakeCheckpointLock,
+			getMockedRuntimeContext(fakeTotalCountOfSubtasks, fakeTndexOfThisSubtask),
+			fakeConfiguration,
+			new KinesisDeserializationSchemaWrapper<>(new SimpleStringSchema()),
+			thrownErrorUnderTest,
+			subscribedShardsStateUnderTest,
+			subscribedStreamsToLastDiscoveredShardIdsStateUnderTest,
+			fakeKinesis);
+
+		this.numElementsCollected = 0;
+	}
+
+	public long getNumOfElementsCollected() {
+		return numElementsCollected;
+	}
+
+	@Override
+	protected KinesisDeserializationSchema<String> getClonedDeserializationSchema() {
+		return new KinesisDeserializationSchemaWrapper<>(new SimpleStringSchema());
+	}
+
+	@Override
+	protected void emitRecordAndUpdateState(String record, long recordTimestamp, int shardStateIndex, SequenceNumber lastSequenceNumber) {
+		synchronized (fakeCheckpointLock) {
+			this.numElementsCollected++;
+			updateState(shardStateIndex, lastSequenceNumber);
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	private static SourceFunction.SourceContext<String> getMockedSourceContext() {
+		return Mockito.mock(SourceFunction.SourceContext.class);
+	}
+
+	private static RuntimeContext getMockedRuntimeContext(final int fakeTotalCountOfSubtasks, final int fakeTndexOfThisSubtask) {
+		RuntimeContext mockedRuntimeContext = Mockito.mock(RuntimeContext.class);
+
+		Mockito.when(mockedRuntimeContext.getNumberOfParallelSubtasks()).thenAnswer(new Answer<Integer>() {
+			@Override
+			public Integer answer(InvocationOnMock invocationOnMock) throws Throwable {
+				return fakeTotalCountOfSubtasks;
+			}
+		});
+
+		Mockito.when(mockedRuntimeContext.getIndexOfThisSubtask()).thenAnswer(new Answer<Integer>() {
+			@Override
+			public Integer answer(InvocationOnMock invocationOnMock) throws Throwable {
+				return fakeTndexOfThisSubtask;
+			}
+		});
+
+		Mockito.when(mockedRuntimeContext.getTaskName()).thenAnswer(new Answer<String>() {
+			@Override
+			public String answer(InvocationOnMock invocationOnMock) throws Throwable {
+				return "Fake Task";
+			}
+		});
+
+		Mockito.when(mockedRuntimeContext.getTaskNameWithSubtasks()).thenAnswer(new Answer<String>() {
+			@Override
+			public String answer(InvocationOnMock invocationOnMock) throws Throwable {
+				return "Fake Task (" + fakeTndexOfThisSubtask + "/" + fakeTotalCountOfSubtasks + ")";
+			}
+		});
+
+		return mockedRuntimeContext;
+	}
+}

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
new file mode 100644
index 0000000..a0795d6
--- /dev/null
+++ b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.rabbitmq;
+
+import java.io.IOException;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.connectors.rabbitmq.common.RMQConnectionConfig;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+
+/**
+ * A Sink for publishing data into RabbitMQ
+ * @param <IN>
+ */
+public class RMQSink<IN> extends RichSinkFunction<IN> {
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(RMQSink.class);
+
+	protected final String queueName;
+	private final RMQConnectionConfig rmqConnectionConfig;
+	protected transient Connection connection;
+	protected transient Channel channel;
+	protected SerializationSchema<IN> schema;
+	private boolean logFailuresOnly = false;
+
+	/**
+	 * @param rmqConnectionConfig The RabbitMQ connection configuration {@link RMQConnectionConfig}.
+	 * @param queueName The queue to publish messages to.
+	 * @param schema A {@link SerializationSchema} for turning the Java objects received into bytes
+     */
+	public RMQSink(RMQConnectionConfig rmqConnectionConfig, String queueName, SerializationSchema<IN> schema) {
+		this.rmqConnectionConfig = rmqConnectionConfig;
+		this.queueName = queueName;
+		this.schema = schema;
+	}
+
+	/**
+	 * Sets up the queue. The default implementation just declares the queue. The user may override
+	 * this method to have a custom setup for the queue (i.e. binding the queue to an exchange or
+	 * defining custom queue parameters)
+	 */
+	protected void setupQueue() throws IOException {
+		channel.queueDeclare(queueName, false, false, false, null);
+	}
+
+	/**
+	 * Defines whether the producer should fail on errors, or only log them.
+	 * If this is set to true, then exceptions will be only logged, if set to false,
+	 * exceptions will be eventually thrown and cause the streaming program to
+	 * fail (and enter recovery).
+	 *
+	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
+	 */
+	public void setLogFailuresOnly(boolean logFailuresOnly) {
+		this.logFailuresOnly = logFailuresOnly;
+	}
+
+
+	@Override
+	public void open(Configuration config) throws Exception {
+		ConnectionFactory factory = rmqConnectionConfig.getConnectionFactory();
+		try {
+			connection = factory.newConnection();
+			channel = connection.createChannel();
+			if (channel == null) {
+				throw new RuntimeException("None of RabbitMQ channels are available");
+			}
+			setupQueue();
+		} catch (IOException e) {
+			throw new RuntimeException("Error while creating the channel", e);
+		}
+	}
+
+	/**
+	 * Called when new data arrives to the sink, and forwards it to RMQ.
+	 *
+	 * @param value
+	 *            The incoming data
+	 */
+	@Override
+	public void invoke(IN value) {
+		try {
+			byte[] msg = schema.serialize(value);
+
+			channel.basicPublish("", queueName, null, msg);
+		} catch (IOException e) {
+			if (logFailuresOnly) {
+				LOG.error("Cannot send RMQ message {} at {}", queueName, rmqConnectionConfig.getHost(), e);
+			} else {
+				throw new RuntimeException("Cannot send RMQ message " + queueName +" at " + rmqConnectionConfig.getHost(), e);
+			}
+		}
+
+	}
+
+	@Override
+	public void close() {
+		IOException t = null;
+		try {
+			channel.close();
+		} catch (IOException e) {
+			t = e;
+		}
+
+		try {
+			connection.close();
+		} catch (IOException e) {
+			if(t != null) {
+				LOG.warn("Both channel and connection closing failed. Logging channel exception and failing with connection exception", t);
+			}
+			t = e;
+		}
+		if(t != null) {
+			throw new RuntimeException("Error while closing RMQ connection with " + queueName
+					+ " at " + rmqConnectionConfig.getHost(), t);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
new file mode 100644
index 0000000..ee9c3b9
--- /dev/null
+++ b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.rabbitmq;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.MessageAcknowledgingSourceBase;
+import org.apache.flink.streaming.api.functions.source.MultipleIdsMessageAcknowledgingSourceBase;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.connectors.rabbitmq.common.RMQConnectionConfig;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.util.Preconditions;
+
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+import com.rabbitmq.client.QueueingConsumer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * RabbitMQ source (consumer) which reads from a queue and acknowledges messages on checkpoints.
+ * When checkpointing is enabled, it guarantees exactly-once processing semantics.
+ *
+ * RabbitMQ requires messages to be acknowledged. On failures, RabbitMQ will re-resend all messages
+ * which have not been acknowledged previously. When a failure occurs directly after a completed
+ * checkpoint, all messages part of this checkpoint might be processed again because they couldn't
+ * be acknowledged before failure. This case is handled by the {@link MessageAcknowledgingSourceBase}
+ * base class which deduplicates the messages using the correlation id.
+ *
+ * RabbitMQ's Delivery Tags do NOT represent unique ids / offsets. That's why the source uses the
+ * Correlation ID in the message properties to check for duplicate messages. Note that the
+ * correlation id has to be set at the producer. If the correlation id is not set, messages may
+ * be produced more than once in corner cases.
+ *
+ * This source can be operated in three different modes:
+ *
+ * 1) Exactly-once (when checkpointed) with RabbitMQ transactions and messages with
+ *    unique correlation IDs.
+ * 2) At-least-once (when checkpointed) with RabbitMQ transactions but no deduplication mechanism
+ *    (correlation id is not set).
+ * 3) No strong delivery guarantees (without checkpointing) with RabbitMQ auto-commit mode.
+ *
+ * Users may overwrite the setupConnectionFactory() method to pass their setup their own
+ * ConnectionFactory in case the constructor parameters are not sufficient.
+ *
+ * @param <OUT> The type of the data read from RabbitMQ.
+ */
+public class RMQSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase<OUT, String, Long>
+	implements ResultTypeQueryable<OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(RMQSource.class);
+
+	private final RMQConnectionConfig rmqConnectionConfig;
+	protected final String queueName;
+	private final boolean usesCorrelationId;
+	protected DeserializationSchema<OUT> schema;
+
+	protected transient Connection connection;
+	protected transient Channel channel;
+	protected transient QueueingConsumer consumer;
+
+	protected transient boolean autoAck;
+
+	private transient volatile boolean running;
+
+	/**
+	 * Creates a new RabbitMQ source with at-least-once message processing guarantee when
+	 * checkpointing is enabled. No strong delivery guarantees when checkpointing is disabled.
+	 * For exactly-once, please use the constructor
+	 * {@link RMQSource#RMQSource(RMQConnectionConfig, String, boolean usesCorrelationId, DeserializationSchema)},
+	 * set {@param usesCorrelationId} to true and enable checkpointing.
+	 * @param rmqConnectionConfig The RabbiMQ connection configuration {@link RMQConnectionConfig}.
+	 * @param queueName  The queue to receive messages from.
+	 * @param deserializationSchema A {@link DeserializationSchema} for turning the bytes received
+	 *               				into Java objects.
+	 */
+	public RMQSource(RMQConnectionConfig rmqConnectionConfig, String queueName,
+					DeserializationSchema<OUT> deserializationSchema) {
+		this(rmqConnectionConfig, queueName, false, deserializationSchema);
+	}
+
+	/**
+	 * Creates a new RabbitMQ source. For exactly-once, you must set the correlation ids of messages
+	 * at the producer. The correlation id must be unique. Otherwise the behavior of the source is
+	 * undefined. In doubt, set {@param usesCorrelationId} to false. When correlation ids are not
+	 * used, this source has at-least-once processing semantics when checkpointing is enabled.
+	 * @param rmqConnectionConfig The RabbiMQ connection configuration {@link RMQConnectionConfig}.
+	 * @param queueName The queue to receive messages from.
+	 * @param usesCorrelationId Whether the messages received are supplied with a <b>unique</b>
+	 *                          id to deduplicate messages (in case of failed acknowledgments).
+	 *                          Only used when checkpointing is enabled.
+	 * @param deserializationSchema A {@link DeserializationSchema} for turning the bytes received
+	 *                              into Java objects.
+	 */
+	public RMQSource(RMQConnectionConfig rmqConnectionConfig,
+					String queueName, boolean usesCorrelationId,DeserializationSchema<OUT> deserializationSchema) {
+		super(String.class);
+		this.rmqConnectionConfig = rmqConnectionConfig;
+		this.queueName = queueName;
+		this.usesCorrelationId = usesCorrelationId;
+		this.schema = deserializationSchema;
+	}
+
+	/**
+	 * Initializes the connection to RMQ with a default connection factory. The user may override
+	 * this method to setup and configure their own ConnectionFactory.
+	 */
+	protected ConnectionFactory setupConnectionFactory() throws Exception {
+		return rmqConnectionConfig.getConnectionFactory();
+	}
+
+	/**
+	 * Sets up the queue. The default implementation just declares the queue. The user may override
+	 * this method to have a custom setup for the queue (i.e. binding the queue to an exchange or
+	 * defining custom queue parameters)
+	 */
+	protected void setupQueue() throws IOException {
+		channel.queueDeclare(queueName, true, false, false, null);
+	}
+
+	@Override
+	public void open(Configuration config) throws Exception {
+		super.open(config);
+		ConnectionFactory factory = setupConnectionFactory();
+		try {
+			connection = factory.newConnection();
+			channel = connection.createChannel();
+			if (channel == null) {
+				throw new RuntimeException("None of RabbitMQ channels are available");
+			}
+			setupQueue();
+			consumer = new QueueingConsumer(channel);
+
+			RuntimeContext runtimeContext = getRuntimeContext();
+			if (runtimeContext instanceof StreamingRuntimeContext
+					&& ((StreamingRuntimeContext) runtimeContext).isCheckpointingEnabled()) {
+				autoAck = false;
+				// enables transaction mode
+				channel.txSelect();
+			} else {
+				autoAck = true;
+			}
+
+			LOG.debug("Starting RabbitMQ source with autoAck status: " + autoAck);
+			channel.basicConsume(queueName, autoAck, consumer);
+
+		} catch (IOException e) {
+			throw new RuntimeException("Cannot create RMQ connection with " + queueName + " at "
+					+ rmqConnectionConfig.getHost(), e);
+		}
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		try {
+			connection.close();
+		} catch (IOException e) {
+			throw new RuntimeException("Error while closing RMQ connection with " + queueName
+				+ " at " + rmqConnectionConfig.getHost(), e);
+		}
+	}
+
+
+	@Override
+	public void run(SourceContext<OUT> ctx) throws Exception {
+		while (running) {
+			QueueingConsumer.Delivery delivery = consumer.nextDelivery();
+
+			synchronized (ctx.getCheckpointLock()) {
+
+				OUT result = schema.deserialize(delivery.getBody());
+
+				if (schema.isEndOfStream(result)) {
+					break;
+				}
+
+				if (!autoAck) {
+					final long deliveryTag = delivery.getEnvelope().getDeliveryTag();
+					if (usesCorrelationId) {
+						final String correlationId = delivery.getProperties().getCorrelationId();
+						Preconditions.checkNotNull(correlationId, "RabbitMQ source was instantiated " +
+							"with usesCorrelationId set to true but a message was received with " +
+							"correlation id set to null!");
+						if (!addId(correlationId)) {
+							// we have already processed this message
+							continue;
+						}
+					}
+					sessionIds.add(deliveryTag);
+				}
+
+				ctx.collect(result);
+			}
+		}
+	}
+
+	@Override
+	public void cancel() {
+		running = false;
+	}
+
+	@Override
+	protected void acknowledgeSessionIDs(List<Long> sessionIds) {
+		try {
+			for (long id : sessionIds) {
+				channel.basicAck(id, false);
+			}
+			channel.txCommit();
+		} catch (IOException e) {
+			throw new RuntimeException("Messages could not be acknowledged during checkpoint creation.", e);
+		}
+	}
+
+	@Override
+	public TypeInformation<OUT> getProducedType() {
+		return schema.getProducedType();
+	}
+}


[03/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/pom.xml b/flink-streaming-connectors/flink-connector-kinesis/pom.xml
deleted file mode 100644
index 29170ad..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/pom.xml
+++ /dev/null
@@ -1,164 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.2-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-kinesis_2.10</artifactId>
-	<name>flink-connector-kinesis</name>
-	<properties>
-		<aws.sdk.version>1.10.71</aws.sdk.version>
-		<aws.kinesis-kcl.version>1.6.2</aws.kinesis-kcl.version>
-		<aws.kinesis-kpl.version>0.10.2</aws.kinesis-kpl.version>
-	</properties>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<!-- Note:
-			The below dependencies are licenced under the Amazon Software License.
-			Flink includes the "flink-connector-kinesis" only as an optional dependency for that reason.
-		-->
-		<dependency>
-			<groupId>com.amazonaws</groupId>
-			<artifactId>aws-java-sdk-kinesis</artifactId>
-			<version>${aws.sdk.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>com.amazonaws</groupId>
-			<artifactId>amazon-kinesis-producer</artifactId>
-			<version>${aws.kinesis-kpl.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>com.amazonaws</groupId>
-			<artifactId>amazon-kinesis-client</artifactId>
-			<version>${aws.kinesis-kcl.version}</version>
-			<!--
-				We're excluding the below from the KCL since we'll only be using the
-				com.amazonaws.services.kinesis.clientlibrary.types.UserRecord class, which will not need these dependencies.
-			-->
-			<exclusions>
-				<exclusion>
-					<groupId>com.amazonaws</groupId>
-					<artifactId>aws-java-sdk-dynamodb</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.amazonaws</groupId>
-					<artifactId>aws-java-sdk-cloudwatch</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>shade-flink</id>
-						<phase>package</phase>
-						<goals>
-							<goal>shade</goal>
-						</goals>
-						<configuration>
-							<promoteTransitiveDependencies>true</promoteTransitiveDependencies>
-							<artifactSet combine.children="append">
-								<includes>
-									<include>com.amazonaws:*</include>
-									<include>com.google.protobuf:*</include>
-								</includes>
-							</artifactSet>
-							<relocations combine.children="override">
-								<!-- DO NOT RELOCATE GUAVA IN THIS PACKAGE -->
-								<relocation>
-									<pattern>org.objectweb.asm</pattern>
-									<shadedPattern>org.apache.flink.shaded.org.objectweb.asm</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>com.google.protobuf</pattern>
-									<shadedPattern>org.apache.flink.kinesis.shaded.com.google.protobuf</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>com.amazonaws</pattern>
-									<shadedPattern>org.apache.flink.kinesis.shaded.com.amazonaws</shadedPattern>
-								</relocation>
-							</relocations>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
deleted file mode 100644
index a62dc10..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
+++ /dev/null
@@ -1,304 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
-import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
-import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
-import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.List;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The Flink Kinesis Consumer is an exactly-once parallel streaming data source that subscribes to multiple AWS Kinesis
- * streams within the same AWS service region, and can handle resharding of streams. Each subtask of the consumer is
- * responsible for fetching data records from multiple Kinesis shards. The number of shards fetched by each subtask will
- * change as shards are closed and created by Kinesis.
- *
- * <p>To leverage Flink's checkpointing mechanics for exactly-once streaming processing guarantees, the Flink Kinesis
- * consumer is implemented with the AWS Java SDK, instead of the officially recommended AWS Kinesis Client Library, for
- * low-level control on the management of stream state. The Flink Kinesis Connector also supports setting the initial
- * starting points of Kinesis streams, namely TRIM_HORIZON and LATEST.</p>
- *
- * @param <T> the type of data emitted
- */
-public class FlinkKinesisConsumer<T> extends RichParallelSourceFunction<T>
-	implements CheckpointedAsynchronously<HashMap<KinesisStreamShard, SequenceNumber>>, ResultTypeQueryable<T> {
-
-	private static final long serialVersionUID = 4724006128720664870L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkKinesisConsumer.class);
-
-	// ------------------------------------------------------------------------
-	//  Consumer properties
-	// ------------------------------------------------------------------------
-
-	/** The names of the Kinesis streams that we will be consuming from */
-	private final List<String> streams;
-
-	/** Properties to parametrize settings such as AWS service region, initial position in stream,
-	 * shard list retrieval behaviours, etc */
-	private final Properties configProps;
-
-	/** User supplied deseriliazation schema to convert Kinesis byte messages to Flink objects */
-	private final KinesisDeserializationSchema<T> deserializer;
-
-	// ------------------------------------------------------------------------
-	//  Runtime state
-	// ------------------------------------------------------------------------
-
-	/** Per-task fetcher for Kinesis data records, where each fetcher pulls data from one or more Kinesis shards */
-	private transient KinesisDataFetcher<T> fetcher;
-
-	/** The sequence numbers in the last state snapshot of this subtask */
-	private transient HashMap<KinesisStreamShard, SequenceNumber> lastStateSnapshot;
-
-	/** The sequence numbers to restore to upon restore from failure */
-	private transient HashMap<KinesisStreamShard, SequenceNumber> sequenceNumsToRestore;
-
-	private volatile boolean running = true;
-
-
-	// ------------------------------------------------------------------------
-	//  Constructors
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new Flink Kinesis Consumer.
-	 *
-	 * <p>The AWS credentials to be used, AWS region of the Kinesis streams, initial position to start streaming
-	 * from are configured with a {@link Properties} instance.</p>
-	 *
-	 * @param stream
-	 *           The single AWS Kinesis stream to read from.
-	 * @param deserializer
-	 *           The deserializer used to convert raw bytes of Kinesis records to Java objects (without key).
-	 * @param configProps
-	 *           The properties used to configure AWS credentials, AWS region, and initial starting position.
-	 */
-	public FlinkKinesisConsumer(String stream, DeserializationSchema<T> deserializer, Properties configProps) {
-		this(stream, new KinesisDeserializationSchemaWrapper<>(deserializer), configProps);
-	}
-
-	/**
-	 * Creates a new Flink Kinesis Consumer.
-	 *
-	 * <p>The AWS credentials to be used, AWS region of the Kinesis streams, initial position to start streaming
-	 * from are configured with a {@link Properties} instance.</p>
-	 *
-	 * @param stream
-	 *           The single AWS Kinesis stream to read from.
-	 * @param deserializer
-	 *           The keyed deserializer used to convert raw bytes of Kinesis records to Java objects.
-	 * @param configProps
-	 *           The properties used to configure AWS credentials, AWS region, and initial starting position.
-	 */
-	public FlinkKinesisConsumer(String stream, KinesisDeserializationSchema<T> deserializer, Properties configProps) {
-		this(Collections.singletonList(stream), deserializer, configProps);
-	}
-
-	/**
-	 * Creates a new Flink Kinesis Consumer.
-	 *
-	 * <p>The AWS credentials to be used, AWS region of the Kinesis streams, initial position to start streaming
-	 * from are configured with a {@link Properties} instance.</p>
-	 *
-	 * @param streams
-	 *           The AWS Kinesis streams to read from.
-	 * @param deserializer
-	 *           The keyed deserializer used to convert raw bytes of Kinesis records to Java objects.
-	 * @param configProps
-	 *           The properties used to configure AWS credentials, AWS region, and initial starting position.
-	 */
-	public FlinkKinesisConsumer(List<String> streams, KinesisDeserializationSchema<T> deserializer, Properties configProps) {
-		checkNotNull(streams, "streams can not be null");
-		checkArgument(streams.size() != 0, "must be consuming at least 1 stream");
-		checkArgument(!streams.contains(""), "stream names cannot be empty Strings");
-		this.streams = streams;
-
-		this.configProps = checkNotNull(configProps, "configProps can not be null");
-
-		// check the configuration properties for any conflicting settings
-		KinesisConfigUtil.validateConsumerConfiguration(this.configProps);
-
-		this.deserializer = checkNotNull(deserializer, "deserializer can not be null");
-
-		if (LOG.isInfoEnabled()) {
-			StringBuilder sb = new StringBuilder();
-			for (String stream : streams) {
-				sb.append(stream).append(", ");
-			}
-			LOG.info("Flink Kinesis Consumer is going to read the following streams: {}", sb.toString());
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Source life cycle
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-
-		// restore to the last known sequence numbers from the latest complete snapshot
-		if (sequenceNumsToRestore != null) {
-			if (LOG.isInfoEnabled()) {
-				LOG.info("Subtask {} is restoring sequence numbers {} from previous checkpointed state",
-					getRuntimeContext().getIndexOfThisSubtask(), sequenceNumsToRestore.toString());
-			}
-
-			// initialize sequence numbers with restored state
-			lastStateSnapshot = sequenceNumsToRestore;
-		} else {
-			// start fresh with empty sequence numbers if there are no snapshots to restore from.
-			lastStateSnapshot = new HashMap<>();
-		}
-	}
-
-	@Override
-	public void run(SourceContext<T> sourceContext) throws Exception {
-
-		// all subtasks will run a fetcher, regardless of whether or not the subtask will initially have
-		// shards to subscribe to; fetchers will continuously poll for changes in the shard list, so all subtasks
-		// can potentially have new shards to subscribe to later on
-		fetcher = new KinesisDataFetcher<>(
-			streams, sourceContext, getRuntimeContext(), configProps, deserializer);
-
-		boolean isRestoringFromFailure = (sequenceNumsToRestore != null);
-		fetcher.setIsRestoringFromFailure(isRestoringFromFailure);
-
-		// if we are restoring from a checkpoint, we iterate over the restored
-		// state and accordingly seed the fetcher with subscribed shards states
-		if (isRestoringFromFailure) {
-			for (Map.Entry<KinesisStreamShard, SequenceNumber> restored : lastStateSnapshot.entrySet()) {
-				fetcher.advanceLastDiscoveredShardOfStream(
-					restored.getKey().getStreamName(), restored.getKey().getShard().getShardId());
-
-				if (LOG.isInfoEnabled()) {
-					LOG.info("Subtask {} is seeding the fetcher with restored shard {}," +
-							" starting state set to the restored sequence number {}",
-						getRuntimeContext().getIndexOfThisSubtask(), restored.getKey().toString(), restored.getValue());
-				}
-				fetcher.registerNewSubscribedShardState(
-					new KinesisStreamShardState(restored.getKey(), restored.getValue()));
-			}
-		}
-
-		// check that we are running before starting the fetcher
-		if (!running) {
-			return;
-		}
-
-		// start the fetcher loop. The fetcher will stop running only when cancel() or
-		// close() is called, or an error is thrown by threads created by the fetcher
-		fetcher.runFetcher();
-
-		// check that the fetcher has terminated before fully closing
-		fetcher.awaitTermination();
-		sourceContext.close();
-	}
-
-	@Override
-	public void cancel() {
-		running = false;
-
-		KinesisDataFetcher fetcher = this.fetcher;
-		this.fetcher = null;
-
-		// this method might be called before the subtask actually starts running,
-		// so we must check if the fetcher is actually created
-		if (fetcher != null) {
-			try {
-				// interrupt the fetcher of any work
-				fetcher.shutdownFetcher();
-				fetcher.awaitTermination();
-			} catch (Exception e) {
-				LOG.warn("Error while closing Kinesis data fetcher", e);
-			}
-		}
-	}
-
-	@Override
-	public void close() throws Exception {
-		cancel();
-		super.close();
-	}
-
-	@Override
-	public TypeInformation<T> getProducedType() {
-		return deserializer.getProducedType();
-	}
-
-	// ------------------------------------------------------------------------
-	//  State Snapshot & Restore
-	// ------------------------------------------------------------------------
-
-	@Override
-	public HashMap<KinesisStreamShard, SequenceNumber> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-		if (lastStateSnapshot == null) {
-			LOG.debug("snapshotState() requested on not yet opened source; returning null.");
-			return null;
-		}
-
-		if (fetcher == null) {
-			LOG.debug("snapshotState() requested on not yet running source; returning null.");
-			return null;
-		}
-
-		if (!running) {
-			LOG.debug("snapshotState() called on closed source; returning null.");
-			return null;
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Snapshotting state ...");
-		}
-
-		lastStateSnapshot = fetcher.snapshotState();
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Snapshotted state, last processed sequence numbers: {}, checkpoint id: {}, timestamp: {}",
-				lastStateSnapshot.toString(), checkpointId, checkpointTimestamp);
-		}
-
-		return lastStateSnapshot;
-	}
-
-	@Override
-	public void restoreState(HashMap<KinesisStreamShard, SequenceNumber> restoredState) throws Exception {
-		sequenceNumsToRestore = restoredState;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
deleted file mode 100644
index 579bd6b..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
+++ /dev/null
@@ -1,292 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis;
-
-import com.amazonaws.services.kinesis.producer.Attempt;
-import com.amazonaws.services.kinesis.producer.KinesisProducer;
-import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration;
-import com.amazonaws.services.kinesis.producer.UserRecordFailedException;
-import com.amazonaws.services.kinesis.producer.UserRecordResult;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
-import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema;
-import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
-import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flink.util.PropertiesUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Objects;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The FlinkKinesisProducer allows to produce from a Flink DataStream into Kinesis.
- *
- * @param <OUT> Data type to produce into Kinesis Streams
- */
-public class FlinkKinesisProducer<OUT> extends RichSinkFunction<OUT> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkKinesisProducer.class);
-
-	/** Properties to parametrize settings such as AWS service region, access key etc. */
-	private final Properties configProps;
-
-	/* Flag controlling the error behavior of the producer */
-	private boolean failOnError = false;
-
-	/* Name of the default stream to produce to. Can be overwritten by the serialization schema */
-	private String defaultStream;
-
-	/* Default partition id. Can be overwritten by the serialization schema */
-	private String defaultPartition;
-
-	/* Schema for turning the OUT type into a byte array. */
-	private final KinesisSerializationSchema<OUT> schema;
-
-	/* Optional custom partitioner */
-	private KinesisPartitioner<OUT> customPartitioner = null;
-
-
-	// --------------------------- Runtime fields ---------------------------
-
-
-	/* Our Kinesis instance for each parallel Flink sink */
-	private transient KinesisProducer producer;
-
-	/* Callback handling failures */
-	private transient FutureCallback<UserRecordResult> callback;
-
-	/* Field for async exception */
-	private transient volatile Throwable thrownException;
-
-
-	// --------------------------- Initialization and configuration  ---------------------------
-
-
-	/**
-	 * Create a new FlinkKinesisProducer.
-	 * This is a constructor supporting Flink's {@see SerializationSchema}.
-	 *
-	 * @param schema Serialization schema for the data type
-	 * @param configProps The properties used to configure AWS credentials and AWS region
-	 */
-	public FlinkKinesisProducer(final SerializationSchema<OUT> schema, Properties configProps) {
-
-		// create a simple wrapper for the serialization schema
-		this(new KinesisSerializationSchema<OUT>() {
-			@Override
-			public ByteBuffer serialize(OUT element) {
-				// wrap into ByteBuffer
-				return ByteBuffer.wrap(schema.serialize(element));
-			}
-			// use default stream and hash key
-			@Override
-			public String getTargetStream(OUT element) {
-				return null;
-			}
-		}, configProps);
-	}
-
-	/**
-	 * Create a new FlinkKinesisProducer.
-	 * This is a constructor supporting {@see KinesisSerializationSchema}.
-	 *
-	 * @param schema Kinesis serialization schema for the data type
-	 * @param configProps The properties used to configure AWS credentials and AWS region
-	 */
-	public FlinkKinesisProducer(KinesisSerializationSchema<OUT> schema, Properties configProps) {
-		this.configProps = checkNotNull(configProps, "configProps can not be null");
-
-		// check the configuration properties for any conflicting settings
-		KinesisConfigUtil.validateProducerConfiguration(this.configProps);
-
-		ClosureCleaner.ensureSerializable(Objects.requireNonNull(schema));
-		this.schema = schema;
-	}
-
-	/**
-	 * If set to true, the producer will immediately fail with an exception on any error.
-	 * Otherwise, the errors are logged and the producer goes on.
-	 *
-	 * @param failOnError Error behavior flag
-	 */
-	public void setFailOnError(boolean failOnError) {
-		this.failOnError = failOnError;
-	}
-
-	/**
-	 * Set a default stream name.
-	 * @param defaultStream Name of the default Kinesis stream
-	 */
-	public void setDefaultStream(String defaultStream) {
-		this.defaultStream = defaultStream;
-	}
-
-	/**
-	 * Set default partition id
-	 * @param defaultPartition Name of the default partition
-	 */
-	public void setDefaultPartition(String defaultPartition) {
-		this.defaultPartition = defaultPartition;
-	}
-
-	public void setCustomPartitioner(KinesisPartitioner<OUT> partitioner) {
-		Objects.requireNonNull(partitioner);
-		ClosureCleaner.ensureSerializable(partitioner);
-		this.customPartitioner = partitioner;
-	}
-
-
-	// --------------------------- Lifecycle methods ---------------------------
-
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-
-		KinesisProducerConfiguration producerConfig = new KinesisProducerConfiguration();
-
-		producerConfig.setRegion(configProps.getProperty(ProducerConfigConstants.AWS_REGION));
-		producerConfig.setCredentialsProvider(AWSUtil.getCredentialsProvider(configProps));
-		if (configProps.containsKey(ProducerConfigConstants.COLLECTION_MAX_COUNT)) {
-			producerConfig.setCollectionMaxCount(PropertiesUtil.getLong(configProps,
-					ProducerConfigConstants.COLLECTION_MAX_COUNT, producerConfig.getCollectionMaxCount(), LOG));
-		}
-		if (configProps.containsKey(ProducerConfigConstants.AGGREGATION_MAX_COUNT)) {
-			producerConfig.setAggregationMaxCount(PropertiesUtil.getLong(configProps,
-					ProducerConfigConstants.AGGREGATION_MAX_COUNT, producerConfig.getAggregationMaxCount(), LOG));
-		}
-
-		producer = new KinesisProducer(producerConfig);
-		callback = new FutureCallback<UserRecordResult>() {
-			@Override
-			public void onSuccess(UserRecordResult result) {
-				if (!result.isSuccessful()) {
-					if(failOnError) {
-						thrownException = new RuntimeException("Record was not sent successful");
-					} else {
-						LOG.warn("Record was not sent successful");
-					}
-				}
-			}
-
-			@Override
-			public void onFailure(Throwable t) {
-				if (failOnError) {
-					thrownException = t;
-				} else {
-					LOG.warn("An exception occurred while processing a record", t);
-				}
-			}
-		};
-
-		if (this.customPartitioner != null) {
-			this.customPartitioner.initialize(getRuntimeContext().getIndexOfThisSubtask(), getRuntimeContext().getNumberOfParallelSubtasks());
-		}
-
-		LOG.info("Started Kinesis producer instance for region '{}'", producerConfig.getRegion());
-	}
-
-	@Override
-	public void invoke(OUT value) throws Exception {
-		if (this.producer == null) {
-			throw new RuntimeException("Kinesis producer has been closed");
-		}
-		if (thrownException != null) {
-			String errorMessages = "";
-			if (thrownException instanceof UserRecordFailedException) {
-				List<Attempt> attempts = ((UserRecordFailedException) thrownException).getResult().getAttempts();
-				for (Attempt attempt: attempts) {
-					if (attempt.getErrorMessage() != null) {
-						errorMessages += attempt.getErrorMessage() +"\n";
-					}
-				}
-			}
-			if (failOnError) {
-				throw new RuntimeException("An exception was thrown while processing a record: " + errorMessages, thrownException);
-			} else {
-				LOG.warn("An exception was thrown while processing a record: {}", thrownException, errorMessages);
-				thrownException = null; // reset
-			}
-		}
-
-		String stream = defaultStream;
-		String partition = defaultPartition;
-
-		ByteBuffer serialized = schema.serialize(value);
-
-		// maybe set custom stream
-		String customStream = schema.getTargetStream(value);
-		if (customStream != null) {
-			stream = customStream;
-		}
-
-		String explicitHashkey = null;
-		// maybe set custom partition
-		if (customPartitioner != null) {
-			partition = customPartitioner.getPartitionId(value);
-			explicitHashkey = customPartitioner.getExplicitHashKey(value);
-		}
-
-		if (stream == null) {
-			if (failOnError) {
-				throw new RuntimeException("No target stream set");
-			} else {
-				LOG.warn("No target stream set. Skipping record");
-				return;
-			}
-		}
-
-		ListenableFuture<UserRecordResult> cb = producer.addUserRecord(stream, partition, explicitHashkey, serialized);
-		Futures.addCallback(cb, callback);
-	}
-
-	@Override
-	public void close() throws Exception {
-		LOG.info("Closing producer");
-		super.close();
-		KinesisProducer kp = this.producer;
-		this.producer = null;
-		if (kp != null) {
-			LOG.info("Flushing outstanding {} records", kp.getOutstandingRecordsCount());
-			// try to flush all outstanding records
-			while (kp.getOutstandingRecordsCount() > 0) {
-				kp.flush();
-				try {
-					Thread.sleep(500);
-				} catch (InterruptedException e) {
-					LOG.warn("Flushing was interrupted.");
-					// stop the blocking flushing and destroy producer immediately
-					break;
-				}
-			}
-			LOG.info("Flushing done. Destroying producer instance.");
-			kp.destroy();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java
deleted file mode 100644
index bd23abe..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.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.kinesis;
-
-
-import java.io.Serializable;
-
-public abstract class KinesisPartitioner<T> implements Serializable {
-
-	/**
-	 * Return a partition id based on the input
-	 * @param element Element to partition
-	 * @return A string representing the partition id
-	 */
-	public abstract String getPartitionId(T element);
-
-	/**
-	 * Optional method for setting an explicit hash key
-	 * @param element Element to get the hash key for
-	 * @return the hash key for the element
-	 */
-	public String getExplicitHashKey(T element) {
-		return null;
-	}
-
-	/**
-	 * Optional initializer.
-	 *
-	 * @param indexOfThisSubtask Index of this partitioner instance
-	 * @param numberOfParallelSubtasks Total number of parallel instances
-	 */
-	public void initialize(int indexOfThisSubtask, int numberOfParallelSubtasks) {
-		//
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java
deleted file mode 100644
index 01d4f00..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.config;
-
-import com.amazonaws.auth.AWSCredentialsProvider;
-
-/**
- * Configuration keys for AWS service usage
- */
-public class AWSConfigConstants {
-
-	/**
-	 * Possible configuration values for the type of credential provider to use when accessing AWS Kinesis.
-	 * Internally, a corresponding implementation of {@link AWSCredentialsProvider} will be used.
-	 */
-	public enum CredentialProvider {
-
-		/** Look for the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY to create AWS credentials */
-		ENV_VAR,
-
-		/** Look for Java system properties aws.accessKeyId and aws.secretKey to create AWS credentials */
-		SYS_PROP,
-
-		/** Use a AWS credentials profile file to create the AWS credentials */
-		PROFILE,
-
-		/** Simply create AWS credentials by supplying the AWS access key ID and AWS secret key in the configuration properties */
-		BASIC,
-
-		/** A credentials provider chain will be used that searches for credentials in this order: ENV_VARS, SYS_PROPS, PROFILE in the AWS instance metadata **/
-		AUTO,
-	}
-
-	/** The AWS region of the Kinesis streams to be pulled ("us-east-1" is used if not set) */
-	public static final String AWS_REGION = "aws.region";
-
-	/** The AWS access key ID to use when setting credentials provider type to BASIC */
-	public static final String AWS_ACCESS_KEY_ID = "aws.credentials.provider.basic.accesskeyid";
-
-	/** The AWS secret key to use when setting credentials provider type to BASIC */
-	public static final String AWS_SECRET_ACCESS_KEY = "aws.credentials.provider.basic.secretkey";
-
-	/** The credential provider type to use when AWS credentials are required (BASIC is used if not set)*/
-	public static final String AWS_CREDENTIALS_PROVIDER = "aws.credentials.provider";
-
-	/** Optional configuration for profile path if credential provider type is set to be PROFILE */
-	public static final String AWS_PROFILE_PATH = "aws.credentials.provider.profile.path";
-
-	/** Optional configuration for profile name if credential provider type is set to be PROFILE */
-	public static final String AWS_PROFILE_NAME = "aws.credentials.provider.profile.name";
-
-	/** The AWS endpoint for Kinesis (derived from the AWS region setting if not set) */
-	public static final String AWS_ENDPOINT = "aws.endpoint";
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java
deleted file mode 100644
index 76c20ed..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.config;
-
-import com.amazonaws.services.kinesis.model.ShardIteratorType;
-import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
-import org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer;
-import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
-
-/**
- * Optional consumer specific configuration keys and default values for {@link FlinkKinesisConsumer}
- */
-public class ConsumerConfigConstants extends AWSConfigConstants {
-
-	/**
-	 * The initial position to start reading shards from. This will affect the {@link ShardIteratorType} used
-	 * when the consumer tasks retrieve the first shard iterator for each Kinesis shard.
-	 */
-	public enum InitialPosition {
-
-		/** Start reading from the earliest possible record in the stream (excluding expired data records) */
-		TRIM_HORIZON(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM),
-
-		/** Start reading from the latest incoming record */
-		LATEST(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM);
-
-		private SentinelSequenceNumber sentinelSequenceNumber;
-
-		InitialPosition(SentinelSequenceNumber sentinelSequenceNumber) {
-			this.sentinelSequenceNumber = sentinelSequenceNumber;
-		}
-
-		public SentinelSequenceNumber toSentinelSequenceNumber() {
-			return this.sentinelSequenceNumber;
-		}
-	}
-
-	/** The initial position to start reading Kinesis streams from (LATEST is used if not set) */
-	public static final String STREAM_INITIAL_POSITION = "flink.stream.initpos";
-
-	/** The base backoff time between each describeStream attempt */
-	public static final String STREAM_DESCRIBE_BACKOFF_BASE = "flink.stream.describe.backoff.base";
-
-	/** The maximum backoff time between each describeStream attempt */
-	public static final String STREAM_DESCRIBE_BACKOFF_MAX = "flink.stream.describe.backoff.max";
-
-	/** The power constant for exponential backoff between each describeStream attempt */
-	public static final String STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = "flink.stream.describe.backoff.expconst";
-
-	/** The maximum number of records to try to get each time we fetch records from a AWS Kinesis shard */
-	public static final String SHARD_GETRECORDS_MAX = "flink.shard.getrecords.maxrecordcount";
-
-	/** The maximum number of getRecords attempts if we get ProvisionedThroughputExceededException */
-	public static final String SHARD_GETRECORDS_RETRIES = "flink.shard.getrecords.maxretries";
-
-	/** The base backoff time between getRecords attempts if we get a ProvisionedThroughputExceededException */
-	public static final String SHARD_GETRECORDS_BACKOFF_BASE = "flink.shard.getrecords.backoff.base";
-
-	/** The maximum backoff time between getRecords attempts if we get a ProvisionedThroughputExceededException */
-	public static final String SHARD_GETRECORDS_BACKOFF_MAX = "flink.shard.getrecords.backoff.max";
-
-	/** The power constant for exponential backoff between each getRecords attempt */
-	public static final String SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT = "flink.shard.getrecords.backoff.expconst";
-
-	/** The interval between each getRecords request to a AWS Kinesis shard in milliseconds */
-	public static final String SHARD_GETRECORDS_INTERVAL_MILLIS = "flink.shard.getrecords.intervalmillis";
-
-	/** The maximum number of getShardIterator attempts if we get ProvisionedThroughputExceededException */
-	public static final String SHARD_GETITERATOR_RETRIES = "flink.shard.getiterator.maxretries";
-
-	/** The base backoff time between getShardIterator attempts if we get a ProvisionedThroughputExceededException */
-	public static final String SHARD_GETITERATOR_BACKOFF_BASE = "flink.shard.getiterator.backoff.base";
-
-	/** The maximum backoff time between getShardIterator attempts if we get a ProvisionedThroughputExceededException */
-	public static final String SHARD_GETITERATOR_BACKOFF_MAX = "flink.shard.getiterator.backoff.max";
-
-	/** The power constant for exponential backoff between each getShardIterator attempt */
-	public static final String SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT = "flink.shard.getiterator.backoff.expconst";
-
-	/** The interval between each attempt to discover new shards */
-	public static final String SHARD_DISCOVERY_INTERVAL_MILLIS = "flink.shard.discovery.intervalmillis";
-
-	// ------------------------------------------------------------------------
-	//  Default values for consumer configuration
-	// ------------------------------------------------------------------------
-
-	public static final String DEFAULT_STREAM_INITIAL_POSITION = InitialPosition.LATEST.toString();
-
-	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 1000L;
-
-	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX = 5000L;
-
-	public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
-
-	public static final int DEFAULT_SHARD_GETRECORDS_MAX = 100;
-
-	public static final int DEFAULT_SHARD_GETRECORDS_RETRIES = 3;
-
-	public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE = 300L;
-
-	public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX = 1000L;
-
-	public static final double DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
-
-	public static final long DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS = 0L;
-
-	public static final int DEFAULT_SHARD_GETITERATOR_RETRIES = 3;
-
-	public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE = 300L;
-
-	public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX = 1000L;
-
-	public static final double DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
-
-	public static final long DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS = 10000L;
-
-	/**
-	 * To avoid shard iterator expires in {@link ShardConsumer}s, the value for the configured
-	 * getRecords interval can not exceed 5 minutes, which is the expire time for retrieved iterators.
-	 */
-	public static final long MAX_SHARD_GETRECORDS_INTERVAL_MILLIS = 300000L;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
deleted file mode 100644
index 1edddfc..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kinesis.config;
-
-import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
-
-/**
- * Optional producer specific configuration keys for {@link FlinkKinesisProducer}
- */
-public class ProducerConfigConstants extends AWSConfigConstants {
-
-	/** Maximum number of items to pack into an PutRecords request. **/
-	public static final String COLLECTION_MAX_COUNT = "aws.producer.collectionMaxCount";
-
-	/** Maximum number of items to pack into an aggregated record. **/
-	public static final String AGGREGATION_MAX_COUNT = "aws.producer.aggregationMaxCount";
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java
deleted file mode 100644
index 55668c6..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kinesis.examples;
-
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
-import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-
-import java.util.Properties;
-
-/**
- * This is an example on how to consume data from Kinesis
- */
-public class ConsumeFromKinesis {
-
-	public static void main(String[] args) throws Exception {
-		ParameterTool pt = ParameterTool.fromArgs(args);
-
-		StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
-		see.setParallelism(1);
-
-		Properties kinesisConsumerConfig = new Properties();
-		kinesisConsumerConfig.setProperty(ConsumerConfigConstants.AWS_REGION, pt.getRequired("region"));
-		kinesisConsumerConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accesskey"));
-		kinesisConsumerConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretkey"));
-
-		DataStream<String> kinesis = see.addSource(new FlinkKinesisConsumer<>(
-			"flink-test",
-			new SimpleStringSchema(),
-			kinesisConsumerConfig));
-
-		kinesis.print();
-
-		see.execute();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java
deleted file mode 100644
index d178137..0000000
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kinesis.examples;
-
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
-import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-
-import java.util.Properties;
-
-/**
- * This is an example on how to produce data into Kinesis
- */
-public class ProduceIntoKinesis {
-
-	public static void main(String[] args) throws Exception {
-		ParameterTool pt = ParameterTool.fromArgs(args);
-
-		StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
-		see.setParallelism(1);
-
-		DataStream<String> simpleStringStream = see.addSource(new EventsGenerator());
-
-		Properties kinesisProducerConfig = new Properties();
-		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region"));
-		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
-		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
-
-		FlinkKinesisProducer<String> kinesis = new FlinkKinesisProducer<>(
-				new SimpleStringSchema(), kinesisProducerConfig);
-
-		kinesis.setFailOnError(true);
-		kinesis.setDefaultStream("flink-test");
-		kinesis.setDefaultPartition("0");
-
-		simpleStringStream.addSink(kinesis);
-
-		see.execute();
-	}
-
-	public static class EventsGenerator implements SourceFunction<String> {
-		private boolean running = true;
-
-		@Override
-		public void run(SourceContext<String> ctx) throws Exception {
-			long seq = 0;
-			while(running) {
-				Thread.sleep(10);
-				ctx.collect((seq++) + "-" + RandomStringUtils.randomAlphabetic(12));
-			}
-		}
-
-		@Override
-		public void cancel() {
-			running = false;
-		}
-	}
-}


[28/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml
new file mode 100644
index 0000000..9e0c7e8
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/pom.xml
@@ -0,0 +1,164 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-kinesis_2.10</artifactId>
+	<name>flink-connector-kinesis</name>
+	<properties>
+		<aws.sdk.version>1.10.71</aws.sdk.version>
+		<aws.kinesis-kcl.version>1.6.2</aws.kinesis-kcl.version>
+		<aws.kinesis-kpl.version>0.10.2</aws.kinesis-kpl.version>
+	</properties>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Note:
+			The below dependencies are licenced under the Amazon Software License.
+			Flink includes the "flink-connector-kinesis" only as an optional dependency for that reason.
+		-->
+		<dependency>
+			<groupId>com.amazonaws</groupId>
+			<artifactId>aws-java-sdk-kinesis</artifactId>
+			<version>${aws.sdk.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>com.amazonaws</groupId>
+			<artifactId>amazon-kinesis-producer</artifactId>
+			<version>${aws.kinesis-kpl.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>com.amazonaws</groupId>
+			<artifactId>amazon-kinesis-client</artifactId>
+			<version>${aws.kinesis-kcl.version}</version>
+			<!--
+				We're excluding the below from the KCL since we'll only be using the
+				com.amazonaws.services.kinesis.clientlibrary.types.UserRecord class, which will not need these dependencies.
+			-->
+			<exclusions>
+				<exclusion>
+					<groupId>com.amazonaws</groupId>
+					<artifactId>aws-java-sdk-dynamodb</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.amazonaws</groupId>
+					<artifactId>aws-java-sdk-cloudwatch</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<promoteTransitiveDependencies>true</promoteTransitiveDependencies>
+							<artifactSet combine.children="append">
+								<includes>
+									<include>com.amazonaws:*</include>
+									<include>com.google.protobuf:*</include>
+								</includes>
+							</artifactSet>
+							<relocations combine.children="override">
+								<!-- DO NOT RELOCATE GUAVA IN THIS PACKAGE -->
+								<relocation>
+									<pattern>org.objectweb.asm</pattern>
+									<shadedPattern>org.apache.flink.shaded.org.objectweb.asm</shadedPattern>
+								</relocation>
+								<relocation>
+									<pattern>com.google.protobuf</pattern>
+									<shadedPattern>org.apache.flink.kinesis.shaded.com.google.protobuf</shadedPattern>
+								</relocation>
+								<relocation>
+									<pattern>com.amazonaws</pattern>
+									<shadedPattern>org.apache.flink.kinesis.shaded.com.amazonaws</shadedPattern>
+								</relocation>
+							</relocations>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
new file mode 100644
index 0000000..a62dc10
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The Flink Kinesis Consumer is an exactly-once parallel streaming data source that subscribes to multiple AWS Kinesis
+ * streams within the same AWS service region, and can handle resharding of streams. Each subtask of the consumer is
+ * responsible for fetching data records from multiple Kinesis shards. The number of shards fetched by each subtask will
+ * change as shards are closed and created by Kinesis.
+ *
+ * <p>To leverage Flink's checkpointing mechanics for exactly-once streaming processing guarantees, the Flink Kinesis
+ * consumer is implemented with the AWS Java SDK, instead of the officially recommended AWS Kinesis Client Library, for
+ * low-level control on the management of stream state. The Flink Kinesis Connector also supports setting the initial
+ * starting points of Kinesis streams, namely TRIM_HORIZON and LATEST.</p>
+ *
+ * @param <T> the type of data emitted
+ */
+public class FlinkKinesisConsumer<T> extends RichParallelSourceFunction<T>
+	implements CheckpointedAsynchronously<HashMap<KinesisStreamShard, SequenceNumber>>, ResultTypeQueryable<T> {
+
+	private static final long serialVersionUID = 4724006128720664870L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkKinesisConsumer.class);
+
+	// ------------------------------------------------------------------------
+	//  Consumer properties
+	// ------------------------------------------------------------------------
+
+	/** The names of the Kinesis streams that we will be consuming from */
+	private final List<String> streams;
+
+	/** Properties to parametrize settings such as AWS service region, initial position in stream,
+	 * shard list retrieval behaviours, etc */
+	private final Properties configProps;
+
+	/** User supplied deseriliazation schema to convert Kinesis byte messages to Flink objects */
+	private final KinesisDeserializationSchema<T> deserializer;
+
+	// ------------------------------------------------------------------------
+	//  Runtime state
+	// ------------------------------------------------------------------------
+
+	/** Per-task fetcher for Kinesis data records, where each fetcher pulls data from one or more Kinesis shards */
+	private transient KinesisDataFetcher<T> fetcher;
+
+	/** The sequence numbers in the last state snapshot of this subtask */
+	private transient HashMap<KinesisStreamShard, SequenceNumber> lastStateSnapshot;
+
+	/** The sequence numbers to restore to upon restore from failure */
+	private transient HashMap<KinesisStreamShard, SequenceNumber> sequenceNumsToRestore;
+
+	private volatile boolean running = true;
+
+
+	// ------------------------------------------------------------------------
+	//  Constructors
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new Flink Kinesis Consumer.
+	 *
+	 * <p>The AWS credentials to be used, AWS region of the Kinesis streams, initial position to start streaming
+	 * from are configured with a {@link Properties} instance.</p>
+	 *
+	 * @param stream
+	 *           The single AWS Kinesis stream to read from.
+	 * @param deserializer
+	 *           The deserializer used to convert raw bytes of Kinesis records to Java objects (without key).
+	 * @param configProps
+	 *           The properties used to configure AWS credentials, AWS region, and initial starting position.
+	 */
+	public FlinkKinesisConsumer(String stream, DeserializationSchema<T> deserializer, Properties configProps) {
+		this(stream, new KinesisDeserializationSchemaWrapper<>(deserializer), configProps);
+	}
+
+	/**
+	 * Creates a new Flink Kinesis Consumer.
+	 *
+	 * <p>The AWS credentials to be used, AWS region of the Kinesis streams, initial position to start streaming
+	 * from are configured with a {@link Properties} instance.</p>
+	 *
+	 * @param stream
+	 *           The single AWS Kinesis stream to read from.
+	 * @param deserializer
+	 *           The keyed deserializer used to convert raw bytes of Kinesis records to Java objects.
+	 * @param configProps
+	 *           The properties used to configure AWS credentials, AWS region, and initial starting position.
+	 */
+	public FlinkKinesisConsumer(String stream, KinesisDeserializationSchema<T> deserializer, Properties configProps) {
+		this(Collections.singletonList(stream), deserializer, configProps);
+	}
+
+	/**
+	 * Creates a new Flink Kinesis Consumer.
+	 *
+	 * <p>The AWS credentials to be used, AWS region of the Kinesis streams, initial position to start streaming
+	 * from are configured with a {@link Properties} instance.</p>
+	 *
+	 * @param streams
+	 *           The AWS Kinesis streams to read from.
+	 * @param deserializer
+	 *           The keyed deserializer used to convert raw bytes of Kinesis records to Java objects.
+	 * @param configProps
+	 *           The properties used to configure AWS credentials, AWS region, and initial starting position.
+	 */
+	public FlinkKinesisConsumer(List<String> streams, KinesisDeserializationSchema<T> deserializer, Properties configProps) {
+		checkNotNull(streams, "streams can not be null");
+		checkArgument(streams.size() != 0, "must be consuming at least 1 stream");
+		checkArgument(!streams.contains(""), "stream names cannot be empty Strings");
+		this.streams = streams;
+
+		this.configProps = checkNotNull(configProps, "configProps can not be null");
+
+		// check the configuration properties for any conflicting settings
+		KinesisConfigUtil.validateConsumerConfiguration(this.configProps);
+
+		this.deserializer = checkNotNull(deserializer, "deserializer can not be null");
+
+		if (LOG.isInfoEnabled()) {
+			StringBuilder sb = new StringBuilder();
+			for (String stream : streams) {
+				sb.append(stream).append(", ");
+			}
+			LOG.info("Flink Kinesis Consumer is going to read the following streams: {}", sb.toString());
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Source life cycle
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+
+		// restore to the last known sequence numbers from the latest complete snapshot
+		if (sequenceNumsToRestore != null) {
+			if (LOG.isInfoEnabled()) {
+				LOG.info("Subtask {} is restoring sequence numbers {} from previous checkpointed state",
+					getRuntimeContext().getIndexOfThisSubtask(), sequenceNumsToRestore.toString());
+			}
+
+			// initialize sequence numbers with restored state
+			lastStateSnapshot = sequenceNumsToRestore;
+		} else {
+			// start fresh with empty sequence numbers if there are no snapshots to restore from.
+			lastStateSnapshot = new HashMap<>();
+		}
+	}
+
+	@Override
+	public void run(SourceContext<T> sourceContext) throws Exception {
+
+		// all subtasks will run a fetcher, regardless of whether or not the subtask will initially have
+		// shards to subscribe to; fetchers will continuously poll for changes in the shard list, so all subtasks
+		// can potentially have new shards to subscribe to later on
+		fetcher = new KinesisDataFetcher<>(
+			streams, sourceContext, getRuntimeContext(), configProps, deserializer);
+
+		boolean isRestoringFromFailure = (sequenceNumsToRestore != null);
+		fetcher.setIsRestoringFromFailure(isRestoringFromFailure);
+
+		// if we are restoring from a checkpoint, we iterate over the restored
+		// state and accordingly seed the fetcher with subscribed shards states
+		if (isRestoringFromFailure) {
+			for (Map.Entry<KinesisStreamShard, SequenceNumber> restored : lastStateSnapshot.entrySet()) {
+				fetcher.advanceLastDiscoveredShardOfStream(
+					restored.getKey().getStreamName(), restored.getKey().getShard().getShardId());
+
+				if (LOG.isInfoEnabled()) {
+					LOG.info("Subtask {} is seeding the fetcher with restored shard {}," +
+							" starting state set to the restored sequence number {}",
+						getRuntimeContext().getIndexOfThisSubtask(), restored.getKey().toString(), restored.getValue());
+				}
+				fetcher.registerNewSubscribedShardState(
+					new KinesisStreamShardState(restored.getKey(), restored.getValue()));
+			}
+		}
+
+		// check that we are running before starting the fetcher
+		if (!running) {
+			return;
+		}
+
+		// start the fetcher loop. The fetcher will stop running only when cancel() or
+		// close() is called, or an error is thrown by threads created by the fetcher
+		fetcher.runFetcher();
+
+		// check that the fetcher has terminated before fully closing
+		fetcher.awaitTermination();
+		sourceContext.close();
+	}
+
+	@Override
+	public void cancel() {
+		running = false;
+
+		KinesisDataFetcher fetcher = this.fetcher;
+		this.fetcher = null;
+
+		// this method might be called before the subtask actually starts running,
+		// so we must check if the fetcher is actually created
+		if (fetcher != null) {
+			try {
+				// interrupt the fetcher of any work
+				fetcher.shutdownFetcher();
+				fetcher.awaitTermination();
+			} catch (Exception e) {
+				LOG.warn("Error while closing Kinesis data fetcher", e);
+			}
+		}
+	}
+
+	@Override
+	public void close() throws Exception {
+		cancel();
+		super.close();
+	}
+
+	@Override
+	public TypeInformation<T> getProducedType() {
+		return deserializer.getProducedType();
+	}
+
+	// ------------------------------------------------------------------------
+	//  State Snapshot & Restore
+	// ------------------------------------------------------------------------
+
+	@Override
+	public HashMap<KinesisStreamShard, SequenceNumber> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+		if (lastStateSnapshot == null) {
+			LOG.debug("snapshotState() requested on not yet opened source; returning null.");
+			return null;
+		}
+
+		if (fetcher == null) {
+			LOG.debug("snapshotState() requested on not yet running source; returning null.");
+			return null;
+		}
+
+		if (!running) {
+			LOG.debug("snapshotState() called on closed source; returning null.");
+			return null;
+		}
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Snapshotting state ...");
+		}
+
+		lastStateSnapshot = fetcher.snapshotState();
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Snapshotted state, last processed sequence numbers: {}, checkpoint id: {}, timestamp: {}",
+				lastStateSnapshot.toString(), checkpointId, checkpointTimestamp);
+		}
+
+		return lastStateSnapshot;
+	}
+
+	@Override
+	public void restoreState(HashMap<KinesisStreamShard, SequenceNumber> restoredState) throws Exception {
+		sequenceNumsToRestore = restoredState;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
new file mode 100644
index 0000000..579bd6b
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis;
+
+import com.amazonaws.services.kinesis.producer.Attempt;
+import com.amazonaws.services.kinesis.producer.KinesisProducer;
+import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration;
+import com.amazonaws.services.kinesis.producer.UserRecordFailedException;
+import com.amazonaws.services.kinesis.producer.UserRecordResult;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.apache.flink.util.PropertiesUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The FlinkKinesisProducer allows to produce from a Flink DataStream into Kinesis.
+ *
+ * @param <OUT> Data type to produce into Kinesis Streams
+ */
+public class FlinkKinesisProducer<OUT> extends RichSinkFunction<OUT> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkKinesisProducer.class);
+
+	/** Properties to parametrize settings such as AWS service region, access key etc. */
+	private final Properties configProps;
+
+	/* Flag controlling the error behavior of the producer */
+	private boolean failOnError = false;
+
+	/* Name of the default stream to produce to. Can be overwritten by the serialization schema */
+	private String defaultStream;
+
+	/* Default partition id. Can be overwritten by the serialization schema */
+	private String defaultPartition;
+
+	/* Schema for turning the OUT type into a byte array. */
+	private final KinesisSerializationSchema<OUT> schema;
+
+	/* Optional custom partitioner */
+	private KinesisPartitioner<OUT> customPartitioner = null;
+
+
+	// --------------------------- Runtime fields ---------------------------
+
+
+	/* Our Kinesis instance for each parallel Flink sink */
+	private transient KinesisProducer producer;
+
+	/* Callback handling failures */
+	private transient FutureCallback<UserRecordResult> callback;
+
+	/* Field for async exception */
+	private transient volatile Throwable thrownException;
+
+
+	// --------------------------- Initialization and configuration  ---------------------------
+
+
+	/**
+	 * Create a new FlinkKinesisProducer.
+	 * This is a constructor supporting Flink's {@see SerializationSchema}.
+	 *
+	 * @param schema Serialization schema for the data type
+	 * @param configProps The properties used to configure AWS credentials and AWS region
+	 */
+	public FlinkKinesisProducer(final SerializationSchema<OUT> schema, Properties configProps) {
+
+		// create a simple wrapper for the serialization schema
+		this(new KinesisSerializationSchema<OUT>() {
+			@Override
+			public ByteBuffer serialize(OUT element) {
+				// wrap into ByteBuffer
+				return ByteBuffer.wrap(schema.serialize(element));
+			}
+			// use default stream and hash key
+			@Override
+			public String getTargetStream(OUT element) {
+				return null;
+			}
+		}, configProps);
+	}
+
+	/**
+	 * Create a new FlinkKinesisProducer.
+	 * This is a constructor supporting {@see KinesisSerializationSchema}.
+	 *
+	 * @param schema Kinesis serialization schema for the data type
+	 * @param configProps The properties used to configure AWS credentials and AWS region
+	 */
+	public FlinkKinesisProducer(KinesisSerializationSchema<OUT> schema, Properties configProps) {
+		this.configProps = checkNotNull(configProps, "configProps can not be null");
+
+		// check the configuration properties for any conflicting settings
+		KinesisConfigUtil.validateProducerConfiguration(this.configProps);
+
+		ClosureCleaner.ensureSerializable(Objects.requireNonNull(schema));
+		this.schema = schema;
+	}
+
+	/**
+	 * If set to true, the producer will immediately fail with an exception on any error.
+	 * Otherwise, the errors are logged and the producer goes on.
+	 *
+	 * @param failOnError Error behavior flag
+	 */
+	public void setFailOnError(boolean failOnError) {
+		this.failOnError = failOnError;
+	}
+
+	/**
+	 * Set a default stream name.
+	 * @param defaultStream Name of the default Kinesis stream
+	 */
+	public void setDefaultStream(String defaultStream) {
+		this.defaultStream = defaultStream;
+	}
+
+	/**
+	 * Set default partition id
+	 * @param defaultPartition Name of the default partition
+	 */
+	public void setDefaultPartition(String defaultPartition) {
+		this.defaultPartition = defaultPartition;
+	}
+
+	public void setCustomPartitioner(KinesisPartitioner<OUT> partitioner) {
+		Objects.requireNonNull(partitioner);
+		ClosureCleaner.ensureSerializable(partitioner);
+		this.customPartitioner = partitioner;
+	}
+
+
+	// --------------------------- Lifecycle methods ---------------------------
+
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+
+		KinesisProducerConfiguration producerConfig = new KinesisProducerConfiguration();
+
+		producerConfig.setRegion(configProps.getProperty(ProducerConfigConstants.AWS_REGION));
+		producerConfig.setCredentialsProvider(AWSUtil.getCredentialsProvider(configProps));
+		if (configProps.containsKey(ProducerConfigConstants.COLLECTION_MAX_COUNT)) {
+			producerConfig.setCollectionMaxCount(PropertiesUtil.getLong(configProps,
+					ProducerConfigConstants.COLLECTION_MAX_COUNT, producerConfig.getCollectionMaxCount(), LOG));
+		}
+		if (configProps.containsKey(ProducerConfigConstants.AGGREGATION_MAX_COUNT)) {
+			producerConfig.setAggregationMaxCount(PropertiesUtil.getLong(configProps,
+					ProducerConfigConstants.AGGREGATION_MAX_COUNT, producerConfig.getAggregationMaxCount(), LOG));
+		}
+
+		producer = new KinesisProducer(producerConfig);
+		callback = new FutureCallback<UserRecordResult>() {
+			@Override
+			public void onSuccess(UserRecordResult result) {
+				if (!result.isSuccessful()) {
+					if(failOnError) {
+						thrownException = new RuntimeException("Record was not sent successful");
+					} else {
+						LOG.warn("Record was not sent successful");
+					}
+				}
+			}
+
+			@Override
+			public void onFailure(Throwable t) {
+				if (failOnError) {
+					thrownException = t;
+				} else {
+					LOG.warn("An exception occurred while processing a record", t);
+				}
+			}
+		};
+
+		if (this.customPartitioner != null) {
+			this.customPartitioner.initialize(getRuntimeContext().getIndexOfThisSubtask(), getRuntimeContext().getNumberOfParallelSubtasks());
+		}
+
+		LOG.info("Started Kinesis producer instance for region '{}'", producerConfig.getRegion());
+	}
+
+	@Override
+	public void invoke(OUT value) throws Exception {
+		if (this.producer == null) {
+			throw new RuntimeException("Kinesis producer has been closed");
+		}
+		if (thrownException != null) {
+			String errorMessages = "";
+			if (thrownException instanceof UserRecordFailedException) {
+				List<Attempt> attempts = ((UserRecordFailedException) thrownException).getResult().getAttempts();
+				for (Attempt attempt: attempts) {
+					if (attempt.getErrorMessage() != null) {
+						errorMessages += attempt.getErrorMessage() +"\n";
+					}
+				}
+			}
+			if (failOnError) {
+				throw new RuntimeException("An exception was thrown while processing a record: " + errorMessages, thrownException);
+			} else {
+				LOG.warn("An exception was thrown while processing a record: {}", thrownException, errorMessages);
+				thrownException = null; // reset
+			}
+		}
+
+		String stream = defaultStream;
+		String partition = defaultPartition;
+
+		ByteBuffer serialized = schema.serialize(value);
+
+		// maybe set custom stream
+		String customStream = schema.getTargetStream(value);
+		if (customStream != null) {
+			stream = customStream;
+		}
+
+		String explicitHashkey = null;
+		// maybe set custom partition
+		if (customPartitioner != null) {
+			partition = customPartitioner.getPartitionId(value);
+			explicitHashkey = customPartitioner.getExplicitHashKey(value);
+		}
+
+		if (stream == null) {
+			if (failOnError) {
+				throw new RuntimeException("No target stream set");
+			} else {
+				LOG.warn("No target stream set. Skipping record");
+				return;
+			}
+		}
+
+		ListenableFuture<UserRecordResult> cb = producer.addUserRecord(stream, partition, explicitHashkey, serialized);
+		Futures.addCallback(cb, callback);
+	}
+
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing producer");
+		super.close();
+		KinesisProducer kp = this.producer;
+		this.producer = null;
+		if (kp != null) {
+			LOG.info("Flushing outstanding {} records", kp.getOutstandingRecordsCount());
+			// try to flush all outstanding records
+			while (kp.getOutstandingRecordsCount() > 0) {
+				kp.flush();
+				try {
+					Thread.sleep(500);
+				} catch (InterruptedException e) {
+					LOG.warn("Flushing was interrupted.");
+					// stop the blocking flushing and destroy producer immediately
+					break;
+				}
+			}
+			LOG.info("Flushing done. Destroying producer instance.");
+			kp.destroy();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java
new file mode 100644
index 0000000..bd23abe
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kinesis;
+
+
+import java.io.Serializable;
+
+public abstract class KinesisPartitioner<T> implements Serializable {
+
+	/**
+	 * Return a partition id based on the input
+	 * @param element Element to partition
+	 * @return A string representing the partition id
+	 */
+	public abstract String getPartitionId(T element);
+
+	/**
+	 * Optional method for setting an explicit hash key
+	 * @param element Element to get the hash key for
+	 * @return the hash key for the element
+	 */
+	public String getExplicitHashKey(T element) {
+		return null;
+	}
+
+	/**
+	 * Optional initializer.
+	 *
+	 * @param indexOfThisSubtask Index of this partitioner instance
+	 * @param numberOfParallelSubtasks Total number of parallel instances
+	 */
+	public void initialize(int indexOfThisSubtask, int numberOfParallelSubtasks) {
+		//
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java
new file mode 100644
index 0000000..01d4f00
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.config;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+
+/**
+ * Configuration keys for AWS service usage
+ */
+public class AWSConfigConstants {
+
+	/**
+	 * Possible configuration values for the type of credential provider to use when accessing AWS Kinesis.
+	 * Internally, a corresponding implementation of {@link AWSCredentialsProvider} will be used.
+	 */
+	public enum CredentialProvider {
+
+		/** Look for the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY to create AWS credentials */
+		ENV_VAR,
+
+		/** Look for Java system properties aws.accessKeyId and aws.secretKey to create AWS credentials */
+		SYS_PROP,
+
+		/** Use a AWS credentials profile file to create the AWS credentials */
+		PROFILE,
+
+		/** Simply create AWS credentials by supplying the AWS access key ID and AWS secret key in the configuration properties */
+		BASIC,
+
+		/** A credentials provider chain will be used that searches for credentials in this order: ENV_VARS, SYS_PROPS, PROFILE in the AWS instance metadata **/
+		AUTO,
+	}
+
+	/** The AWS region of the Kinesis streams to be pulled ("us-east-1" is used if not set) */
+	public static final String AWS_REGION = "aws.region";
+
+	/** The AWS access key ID to use when setting credentials provider type to BASIC */
+	public static final String AWS_ACCESS_KEY_ID = "aws.credentials.provider.basic.accesskeyid";
+
+	/** The AWS secret key to use when setting credentials provider type to BASIC */
+	public static final String AWS_SECRET_ACCESS_KEY = "aws.credentials.provider.basic.secretkey";
+
+	/** The credential provider type to use when AWS credentials are required (BASIC is used if not set)*/
+	public static final String AWS_CREDENTIALS_PROVIDER = "aws.credentials.provider";
+
+	/** Optional configuration for profile path if credential provider type is set to be PROFILE */
+	public static final String AWS_PROFILE_PATH = "aws.credentials.provider.profile.path";
+
+	/** Optional configuration for profile name if credential provider type is set to be PROFILE */
+	public static final String AWS_PROFILE_NAME = "aws.credentials.provider.profile.name";
+
+	/** The AWS endpoint for Kinesis (derived from the AWS region setting if not set) */
+	public static final String AWS_ENDPOINT = "aws.endpoint";
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java
new file mode 100644
index 0000000..76c20ed
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.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.streaming.connectors.kinesis.config;
+
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer;
+import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
+
+/**
+ * Optional consumer specific configuration keys and default values for {@link FlinkKinesisConsumer}
+ */
+public class ConsumerConfigConstants extends AWSConfigConstants {
+
+	/**
+	 * The initial position to start reading shards from. This will affect the {@link ShardIteratorType} used
+	 * when the consumer tasks retrieve the first shard iterator for each Kinesis shard.
+	 */
+	public enum InitialPosition {
+
+		/** Start reading from the earliest possible record in the stream (excluding expired data records) */
+		TRIM_HORIZON(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM),
+
+		/** Start reading from the latest incoming record */
+		LATEST(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM);
+
+		private SentinelSequenceNumber sentinelSequenceNumber;
+
+		InitialPosition(SentinelSequenceNumber sentinelSequenceNumber) {
+			this.sentinelSequenceNumber = sentinelSequenceNumber;
+		}
+
+		public SentinelSequenceNumber toSentinelSequenceNumber() {
+			return this.sentinelSequenceNumber;
+		}
+	}
+
+	/** The initial position to start reading Kinesis streams from (LATEST is used if not set) */
+	public static final String STREAM_INITIAL_POSITION = "flink.stream.initpos";
+
+	/** The base backoff time between each describeStream attempt */
+	public static final String STREAM_DESCRIBE_BACKOFF_BASE = "flink.stream.describe.backoff.base";
+
+	/** The maximum backoff time between each describeStream attempt */
+	public static final String STREAM_DESCRIBE_BACKOFF_MAX = "flink.stream.describe.backoff.max";
+
+	/** The power constant for exponential backoff between each describeStream attempt */
+	public static final String STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = "flink.stream.describe.backoff.expconst";
+
+	/** The maximum number of records to try to get each time we fetch records from a AWS Kinesis shard */
+	public static final String SHARD_GETRECORDS_MAX = "flink.shard.getrecords.maxrecordcount";
+
+	/** The maximum number of getRecords attempts if we get ProvisionedThroughputExceededException */
+	public static final String SHARD_GETRECORDS_RETRIES = "flink.shard.getrecords.maxretries";
+
+	/** The base backoff time between getRecords attempts if we get a ProvisionedThroughputExceededException */
+	public static final String SHARD_GETRECORDS_BACKOFF_BASE = "flink.shard.getrecords.backoff.base";
+
+	/** The maximum backoff time between getRecords attempts if we get a ProvisionedThroughputExceededException */
+	public static final String SHARD_GETRECORDS_BACKOFF_MAX = "flink.shard.getrecords.backoff.max";
+
+	/** The power constant for exponential backoff between each getRecords attempt */
+	public static final String SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT = "flink.shard.getrecords.backoff.expconst";
+
+	/** The interval between each getRecords request to a AWS Kinesis shard in milliseconds */
+	public static final String SHARD_GETRECORDS_INTERVAL_MILLIS = "flink.shard.getrecords.intervalmillis";
+
+	/** The maximum number of getShardIterator attempts if we get ProvisionedThroughputExceededException */
+	public static final String SHARD_GETITERATOR_RETRIES = "flink.shard.getiterator.maxretries";
+
+	/** The base backoff time between getShardIterator attempts if we get a ProvisionedThroughputExceededException */
+	public static final String SHARD_GETITERATOR_BACKOFF_BASE = "flink.shard.getiterator.backoff.base";
+
+	/** The maximum backoff time between getShardIterator attempts if we get a ProvisionedThroughputExceededException */
+	public static final String SHARD_GETITERATOR_BACKOFF_MAX = "flink.shard.getiterator.backoff.max";
+
+	/** The power constant for exponential backoff between each getShardIterator attempt */
+	public static final String SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT = "flink.shard.getiterator.backoff.expconst";
+
+	/** The interval between each attempt to discover new shards */
+	public static final String SHARD_DISCOVERY_INTERVAL_MILLIS = "flink.shard.discovery.intervalmillis";
+
+	// ------------------------------------------------------------------------
+	//  Default values for consumer configuration
+	// ------------------------------------------------------------------------
+
+	public static final String DEFAULT_STREAM_INITIAL_POSITION = InitialPosition.LATEST.toString();
+
+	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 1000L;
+
+	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX = 5000L;
+
+	public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+	public static final int DEFAULT_SHARD_GETRECORDS_MAX = 100;
+
+	public static final int DEFAULT_SHARD_GETRECORDS_RETRIES = 3;
+
+	public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE = 300L;
+
+	public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX = 1000L;
+
+	public static final double DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+	public static final long DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS = 0L;
+
+	public static final int DEFAULT_SHARD_GETITERATOR_RETRIES = 3;
+
+	public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE = 300L;
+
+	public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX = 1000L;
+
+	public static final double DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+	public static final long DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS = 10000L;
+
+	/**
+	 * To avoid shard iterator expires in {@link ShardConsumer}s, the value for the configured
+	 * getRecords interval can not exceed 5 minutes, which is the expire time for retrieved iterators.
+	 */
+	public static final long MAX_SHARD_GETRECORDS_INTERVAL_MILLIS = 300000L;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
new file mode 100644
index 0000000..1edddfc
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.config;
+
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+
+/**
+ * Optional producer specific configuration keys for {@link FlinkKinesisProducer}
+ */
+public class ProducerConfigConstants extends AWSConfigConstants {
+
+	/** Maximum number of items to pack into an PutRecords request. **/
+	public static final String COLLECTION_MAX_COUNT = "aws.producer.collectionMaxCount";
+
+	/** Maximum number of items to pack into an aggregated record. **/
+	public static final String AGGREGATION_MAX_COUNT = "aws.producer.aggregationMaxCount";
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java
new file mode 100644
index 0000000..55668c6
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kinesis.examples;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+
+import java.util.Properties;
+
+/**
+ * This is an example on how to consume data from Kinesis
+ */
+public class ConsumeFromKinesis {
+
+	public static void main(String[] args) throws Exception {
+		ParameterTool pt = ParameterTool.fromArgs(args);
+
+		StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
+		see.setParallelism(1);
+
+		Properties kinesisConsumerConfig = new Properties();
+		kinesisConsumerConfig.setProperty(ConsumerConfigConstants.AWS_REGION, pt.getRequired("region"));
+		kinesisConsumerConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accesskey"));
+		kinesisConsumerConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretkey"));
+
+		DataStream<String> kinesis = see.addSource(new FlinkKinesisConsumer<>(
+			"flink-test",
+			new SimpleStringSchema(),
+			kinesisConsumerConfig));
+
+		kinesis.print();
+
+		see.execute();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java
new file mode 100644
index 0000000..d178137
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kinesis.examples;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+
+import java.util.Properties;
+
+/**
+ * This is an example on how to produce data into Kinesis
+ */
+public class ProduceIntoKinesis {
+
+	public static void main(String[] args) throws Exception {
+		ParameterTool pt = ParameterTool.fromArgs(args);
+
+		StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
+		see.setParallelism(1);
+
+		DataStream<String> simpleStringStream = see.addSource(new EventsGenerator());
+
+		Properties kinesisProducerConfig = new Properties();
+		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region"));
+		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
+		kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
+
+		FlinkKinesisProducer<String> kinesis = new FlinkKinesisProducer<>(
+				new SimpleStringSchema(), kinesisProducerConfig);
+
+		kinesis.setFailOnError(true);
+		kinesis.setDefaultStream("flink-test");
+		kinesis.setDefaultPartition("0");
+
+		simpleStringStream.addSink(kinesis);
+
+		see.execute();
+	}
+
+	public static class EventsGenerator implements SourceFunction<String> {
+		private boolean running = true;
+
+		@Override
+		public void run(SourceContext<String> ctx) throws Exception {
+			long seq = 0;
+			while(running) {
+				Thread.sleep(10);
+				ctx.collect((seq++) + "-" + RandomStringUtils.randomAlphabetic(12));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
new file mode 100644
index 0000000..a06fdca
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
@@ -0,0 +1,679 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.kinesis.internals;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.InitialPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
+import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.util.InstantiationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
+ * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
+ * <ul>
+ *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
+ *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
+ *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
+ *     		  to the same subset of shards even after restoring)</li>
+ *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
+ *     <li>3. subscribe to shards by creating a single thread for each shard</li>
+ * </ul>
+ *
+ * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
+ * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
+ * by multiple threads, these operations should only be done using the handler methods provided in this class.
+ */
+public class KinesisDataFetcher<T> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
+
+	// ------------------------------------------------------------------------
+	//  Consumer-wide settings
+	// ------------------------------------------------------------------------
+
+	/** Configuration properties for the Flink Kinesis Consumer */
+	private final Properties configProps;
+
+	/** The list of Kinesis streams that the consumer is subscribing to */
+	private final List<String> streams;
+
+	/**
+	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
+	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
+	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
+	 */
+	private final KinesisDeserializationSchema<T> deserializationSchema;
+
+	// ------------------------------------------------------------------------
+	//  Subtask-specific settings
+	// ------------------------------------------------------------------------
+
+	/** Runtime context of the subtask that this fetcher was created in */
+	private final RuntimeContext runtimeContext;
+
+	private final int totalNumberOfConsumerSubtasks;
+
+	private final int indexOfThisConsumerSubtask;
+
+	/**
+	 * This flag should be set by {@link FlinkKinesisConsumer} using
+	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
+	 */
+	private boolean isRestoredFromFailure;
+
+	// ------------------------------------------------------------------------
+	//  Executor services to run created threads
+	// ------------------------------------------------------------------------
+
+	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
+	private final ExecutorService shardConsumersExecutor;
+
+	// ------------------------------------------------------------------------
+	//  Managed state, accessed and updated across multiple threads
+	// ------------------------------------------------------------------------
+
+	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
+	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
+	 */
+	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
+
+	/**
+	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
+	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
+	 * the last processed sequence number of subscribed shards as they fetch and process records.
+	 *
+	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
+	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
+	 * all threads must use the following thread-safe methods this class provides to operate on this list:
+	 * <ul>
+	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
+	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
+	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, long, int, SequenceNumber)}</li>
+	 * </ul>
+	 */
+	private final List<KinesisStreamShardState> subscribedShardsState;
+
+	private final SourceFunction.SourceContext<T> sourceContext;
+
+	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
+	private final Object checkpointLock;
+
+	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
+	private final AtomicReference<Throwable> error;
+
+	/** The Kinesis proxy that the fetcher will be using to discover new shards */
+	private final KinesisProxyInterface kinesis;
+
+	/** Thread that executed runFetcher() */
+	private Thread mainThread;
+
+	/**
+	 * The current number of shards that are actively read by this fetcher.
+	 *
+	 * This value is updated in {@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)},
+	 * and {@link KinesisDataFetcher#updateState(int, SequenceNumber)}.
+	 */
+	private final AtomicInteger numberOfActiveShards = new AtomicInteger(0);
+
+	private volatile boolean running = true;
+
+	/**
+	 * Creates a Kinesis Data Fetcher.
+	 *
+	 * @param streams the streams to subscribe to
+	 * @param sourceContext context of the source function
+	 * @param runtimeContext this subtask's runtime context
+	 * @param configProps the consumer configuration properties
+	 * @param deserializationSchema deserialization schema
+	 */
+	public KinesisDataFetcher(List<String> streams,
+							SourceFunction.SourceContext<T> sourceContext,
+							RuntimeContext runtimeContext,
+							Properties configProps,
+							KinesisDeserializationSchema<T> deserializationSchema) {
+		this(streams,
+			sourceContext,
+			sourceContext.getCheckpointLock(),
+			runtimeContext,
+			configProps,
+			deserializationSchema,
+			new AtomicReference<Throwable>(),
+			new LinkedList<KinesisStreamShardState>(),
+			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
+			KinesisProxy.create(configProps));
+	}
+
+	/** This constructor is exposed for testing purposes */
+	protected KinesisDataFetcher(List<String> streams,
+								SourceFunction.SourceContext<T> sourceContext,
+								Object checkpointLock,
+								RuntimeContext runtimeContext,
+								Properties configProps,
+								KinesisDeserializationSchema<T> deserializationSchema,
+								AtomicReference<Throwable> error,
+								LinkedList<KinesisStreamShardState> subscribedShardsState,
+								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
+								KinesisProxyInterface kinesis) {
+		this.streams = checkNotNull(streams);
+		this.configProps = checkNotNull(configProps);
+		this.sourceContext = checkNotNull(sourceContext);
+		this.checkpointLock = checkNotNull(checkpointLock);
+		this.runtimeContext = checkNotNull(runtimeContext);
+		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
+		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
+		this.deserializationSchema = checkNotNull(deserializationSchema);
+		this.kinesis = checkNotNull(kinesis);
+
+		this.error = checkNotNull(error);
+		this.subscribedShardsState = checkNotNull(subscribedShardsState);
+		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
+
+		this.shardConsumersExecutor =
+			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
+	}
+
+	/**
+	 * Starts the fetcher. After starting the fetcher, it can only
+	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
+	 *
+	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
+	 */
+	public void runFetcher() throws Exception {
+
+		// check that we are running before proceeding
+		if (!running) {
+			return;
+		}
+
+		this.mainThread = Thread.currentThread();
+
+		// ------------------------------------------------------------------------
+		//  Procedures before starting the infinite while loop:
+		// ------------------------------------------------------------------------
+
+		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
+		//     and register them to the subscribedShardState list.
+		if (LOG.isDebugEnabled()) {
+			String logFormat = (!isRestoredFromFailure)
+				? "Subtask {} is trying to discover initial shards ..."
+				: "Subtask {} is trying to discover any new shards that were created while the consumer wasn't " +
+				"running due to failure ...";
+
+			LOG.debug(logFormat, indexOfThisConsumerSubtask);
+		}
+		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
+		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
+			// the starting state for new shards created while the consumer wasn't running depends on whether or not
+			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
+			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
+			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
+			InitialPosition initialPosition = InitialPosition.valueOf(configProps.getProperty(
+				ConsumerConfigConstants.STREAM_INITIAL_POSITION, ConsumerConfigConstants.DEFAULT_STREAM_INITIAL_POSITION));
+
+			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
+				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
+				: initialPosition.toSentinelSequenceNumber();
+
+			if (LOG.isInfoEnabled()) {
+				String logFormat = (!isRestoredFromFailure)
+					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
+					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't " +
+					"running due to failure, starting state set as sequence number {}";
+
+				LOG.info(logFormat, indexOfThisConsumerSubtask, shard.toString(), startingStateForNewShard.get());
+			}
+			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
+		}
+
+		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
+		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
+		boolean hasShards = false;
+		StringBuilder streamsWithNoShardsFound = new StringBuilder();
+		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
+			if (streamToLastDiscoveredShardEntry.getValue() != null) {
+				hasShards = true;
+			} else {
+				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
+			}
+		}
+
+		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
+			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
+				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
+		}
+
+		if (!hasShards) {
+			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
+		}
+
+		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
+		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
+		//     consumer using a restored state checkpoint
+		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
+			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
+
+			// only start a consuming thread if the seeded subscribed shard has not been completely read already
+			if (!seededShardState.getLastProcessedSequenceNum().equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
+
+				if (LOG.isInfoEnabled()) {
+					LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}",
+						indexOfThisConsumerSubtask, seededShardState.getKinesisStreamShard().toString(),
+						seededShardState.getLastProcessedSequenceNum(), seededStateIndex);
+					}
+
+				shardConsumersExecutor.submit(
+					new ShardConsumer<>(
+						this,
+						seededStateIndex,
+						subscribedShardsState.get(seededStateIndex).getKinesisStreamShard(),
+						subscribedShardsState.get(seededStateIndex).getLastProcessedSequenceNum()));
+			}
+		}
+
+		// ------------------------------------------------------------------------
+
+		// finally, start the infinite shard discovery and consumer launching loop;
+		// we will escape from this loop only when shutdownFetcher() or stopWithError() is called
+
+		final long discoveryIntervalMillis = Long.valueOf(
+			configProps.getProperty(
+				ConsumerConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS,
+				Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+
+		// FLINK-4341:
+		// For downstream operators that work on time (ex. window operators), we are required to emit a max value watermark
+		// for subtasks that won't continue to have shards to read from unless resharding happens in the future, otherwise
+		// the downstream watermarks would not advance, leading to unbounded accumulating state.
+		//
+		// The side-effect of this limitation is that on resharding, we must fail hard if the newly discovered shard
+		// is to be subscribed by a subtask that has previously emitted a max value watermark, otherwise the watermarks
+		// will be messed up.
+		//
+		// There are 2 cases were we need to either emit a max value watermark, or deliberately fail hard:
+		//  (a) if this subtask has no more shards to read from unless resharding happens in the future, we emit a max
+		//      value watermark. This case is encountered when 1) all previously read shards by this subtask were closed
+		//      due to resharding, 2) when this subtask was initially only subscribed to closed shards while the consumer
+		//      was told to start from TRIM_HORIZON, or 3) there was initially no shards for this subtask to read on startup.
+		//  (b) this subtask has discovered new shards to read from due to a reshard; if this subtask has already emitted
+		//      a max value watermark, we must deliberately fail hard to avoid messing up the watermarks. The new shards
+		//      will be subscribed by this subtask after restore as initial shards on startup.
+		//
+		// TODO: This is a temporary workaround until a min-watermark information service is available in the JobManager
+		// Please see FLINK-4341 for more detail
+
+		boolean emittedMaxValueWatermark = false;
+
+		if (this.numberOfActiveShards.get() == 0) {
+			// FLINK-4341 workaround case (a) - please see the above for details on this case
+			LOG.info("Subtask {} has no initial shards to read on startup; emitting max value watermark ...",
+				indexOfThisConsumerSubtask);
+			sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE));
+			emittedMaxValueWatermark = true;
+		}
+
+		while (running) {
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Subtask {} is trying to discover new shards that were created due to resharding ...",
+					indexOfThisConsumerSubtask);
+			}
+			List<KinesisStreamShard> newShardsDueToResharding = discoverNewShardsToSubscribe();
+
+			// -- NOTE: Potential race condition between newShardsDueToResharding and numberOfActiveShards --
+			// Since numberOfActiveShards is updated by parallel shard consuming threads in updateState(), there exists
+			// a race condition with the currently queried newShardsDueToResharding. Therefore, numberOfActiveShards
+			// may not correctly reflect the discover result in the below case determination. This may lead to incorrect
+			// case determination on the current discovery attempt, but can still be correctly handled on future attempts.
+			//
+			// Although this can be resolved by wrapping the current shard discovery attempt with the below
+			// case determination within a synchronized block on the checkpoint lock for atomicity, there will be
+			// considerable throughput performance regression as shard discovery is a remote call to AWS. Therefore,
+			// since the case determination is a temporary workaround for FLINK-4341, the race condition is tolerable as
+			// we can still eventually handle max value watermark emitting / deliberately failing on successive
+			// discovery attempts.
+
+			if (newShardsDueToResharding.size() == 0 && this.numberOfActiveShards.get() == 0 && !emittedMaxValueWatermark) {
+				// FLINK-4341 workaround case (a) - please see the above for details on this case
+				LOG.info("Subtask {} has completed reading all shards; emitting max value watermark ...",
+					indexOfThisConsumerSubtask);
+				sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE));
+				emittedMaxValueWatermark = true;
+			} else if (newShardsDueToResharding.size() > 0 && emittedMaxValueWatermark) {
+				// FLINK-4341 workaround case (b) - please see the above for details on this case
+				//
+				// Note that in the case where on resharding this subtask ceased to read all of it's previous shards
+				// but new shards is also to be subscribed by this subtask immediately after, emittedMaxValueWatermark
+				// will be false; this allows the fetcher to continue reading the new shards without failing on such cases.
+				// However, due to the race condition mentioned above, we might still fall into case (a) first, and
+				// then (b) on the next discovery attempt. Although the failure is ideally unnecessary, max value
+				// watermark emitting still remains to be correct.
+
+				LOG.warn("Subtask {} has discovered {} new shards to subscribe, but is failing hard to avoid messing" +
+						" up watermarks; the new shards will be subscribed by this subtask after restore ...",
+					indexOfThisConsumerSubtask, newShardsDueToResharding.size());
+				throw new RuntimeException("Deliberate failure to avoid messing up watermarks");
+			}
+
+			for (KinesisStreamShard shard : newShardsDueToResharding) {
+				// since there may be delay in discovering a new shard, all new shards due to
+				// resharding should be read starting from the earliest record possible
+				KinesisStreamShardState newShardState =
+					new KinesisStreamShardState(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get());
+				int newStateIndex = registerNewSubscribedShardState(newShardState);
+
+				if (LOG.isInfoEnabled()) {
+					LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " +
+							"the shard from sequence number {} with ShardConsumer {}",
+						indexOfThisConsumerSubtask, newShardState.getKinesisStreamShard().toString(),
+						newShardState.getLastProcessedSequenceNum(), newStateIndex);
+				}
+
+				shardConsumersExecutor.submit(
+					new ShardConsumer<>(
+						this,
+						newStateIndex,
+						newShardState.getKinesisStreamShard(),
+						newShardState.getLastProcessedSequenceNum()));
+			}
+
+			// we also check if we are running here so that we won't start the discovery sleep
+			// interval if the running flag was set to false during the middle of the while loop
+			if (running && discoveryIntervalMillis != 0) {
+				try {
+					Thread.sleep(discoveryIntervalMillis);
+				} catch (InterruptedException iex) {
+					// the sleep may be interrupted by shutdownFetcher()
+				}
+			}
+		}
+
+		// make sure all resources have been terminated before leaving
+		awaitTermination();
+
+		// any error thrown in the shard consumer threads will be thrown to the main thread
+		Throwable throwable = this.error.get();
+		if (throwable != null) {
+			if (throwable instanceof Exception) {
+				throw (Exception) throwable;
+			} else if (throwable instanceof Error) {
+				throw (Error) throwable;
+			} else {
+				throw new Exception(throwable);
+			}
+		}
+	}
+
+	/**
+	 * Creates a snapshot of the current last processed sequence numbers of each subscribed shard.
+	 *
+	 * @return state snapshot
+	 */
+	public HashMap<KinesisStreamShard, SequenceNumber> snapshotState() {
+		// this method assumes that the checkpoint lock is held
+		assert Thread.holdsLock(checkpointLock);
+
+		HashMap<KinesisStreamShard, SequenceNumber> stateSnapshot = new HashMap<>();
+		for (KinesisStreamShardState shardWithState : subscribedShardsState) {
+			stateSnapshot.put(shardWithState.getKinesisStreamShard(), shardWithState.getLastProcessedSequenceNum());
+		}
+		return stateSnapshot;
+	}
+
+	/**
+	 * Starts shutting down the fetcher. Must be called to allow {@link KinesisDataFetcher#runFetcher()} to complete.
+	 * Once called, the shutdown procedure will be executed and all shard consuming threads will be interrupted.
+	 */
+	public void shutdownFetcher() {
+		running = false;
+		mainThread.interrupt(); // the main thread may be sleeping for the discovery interval
+
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Shutting down the shard consumer threads of subtask {} ...", indexOfThisConsumerSubtask);
+		}
+		shardConsumersExecutor.shutdownNow();
+	}
+
+	/** After calling {@link KinesisDataFetcher#shutdownFetcher()}, this can be called to await the fetcher shutdown */
+	public void awaitTermination() throws InterruptedException {
+		while(!shardConsumersExecutor.isTerminated()) {
+			Thread.sleep(50);
+		}
+	}
+
+	/** Called by created threads to pass on errors. Only the first thrown error is set.
+	 * Once set, the shutdown process will be executed and all shard consuming threads will be interrupted. */
+	protected void stopWithError(Throwable throwable) {
+		if (this.error.compareAndSet(null, throwable)) {
+			shutdownFetcher();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Functions that update the subscribedStreamToLastDiscoveredShardIds state
+	// ------------------------------------------------------------------------
+
+	/** Updates the last discovered shard of a subscribed stream; only updates if the update is valid */
+	public void advanceLastDiscoveredShardOfStream(String stream, String shardId) {
+		String lastSeenShardIdOfStream = this.subscribedStreamsToLastDiscoveredShardIds.get(stream);
+
+		// the update is valid only if the given shard id is greater
+		// than the previous last seen shard id of the stream
+		if (lastSeenShardIdOfStream == null) {
+			// if not previously set, simply put as the last seen shard id
+			this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId);
+		} else if (KinesisStreamShard.compareShardIds(shardId, lastSeenShardIdOfStream) > 0) {
+			this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId);
+		}
+	}
+
+	/**
+	 * A utility function that does the following:
+	 *
+	 * 1. Find new shards for each stream that we haven't seen before
+	 * 2. For each new shard, determine whether this consumer subtask should subscribe to them;
+	 * 	  if yes, it is added to the returned list of shards
+	 * 3. Update the subscribedStreamsToLastDiscoveredShardIds state so that we won't get shards
+	 *    that we have already seen before the next time this function is called
+	 */
+	private List<KinesisStreamShard> discoverNewShardsToSubscribe() throws InterruptedException {
+
+		List<KinesisStreamShard> newShardsToSubscribe = new LinkedList<>();
+
+		GetShardListResult shardListResult = kinesis.getShardList(subscribedStreamsToLastDiscoveredShardIds);
+		if (shardListResult.hasRetrievedShards()) {
+			Set<String> streamsWithNewShards = shardListResult.getStreamsWithRetrievedShards();
+
+			for (String stream : streamsWithNewShards) {
+				List<KinesisStreamShard> newShardsOfStream = shardListResult.getRetrievedShardListOfStream(stream);
+				for (KinesisStreamShard newShard : newShardsOfStream) {
+					if (isThisSubtaskShouldSubscribeTo(newShard, totalNumberOfConsumerSubtasks, indexOfThisConsumerSubtask)) {
+						newShardsToSubscribe.add(newShard);
+					}
+				}
+
+				advanceLastDiscoveredShardOfStream(
+					stream, shardListResult.getLastSeenShardOfStream(stream).getShard().getShardId());
+			}
+		}
+
+		return newShardsToSubscribe;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Functions to get / set information about the consumer
+	// ------------------------------------------------------------------------
+
+	public void setIsRestoringFromFailure(boolean bool) {
+		this.isRestoredFromFailure = bool;
+	}
+
+	protected Properties getConsumerConfiguration() {
+		return configProps;
+	}
+
+	protected KinesisDeserializationSchema<T> getClonedDeserializationSchema() {
+		try {
+			return InstantiationUtil.clone(deserializationSchema, runtimeContext.getUserCodeClassLoader());
+		} catch (IOException | ClassNotFoundException ex) {
+			// this really shouldn't happen; simply wrap it around a runtime exception
+			throw new RuntimeException(ex);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Thread-safe operations for record emitting and shard state updating
+	//  that assure atomicity with respect to the checkpoint lock
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Atomic operation to collect a record and update state to the sequence number of the record.
+	 * This method is called by {@link ShardConsumer}s.
+	 *
+	 * @param record the record to collect
+	 * @param recordTimestamp timestamp to attach to the collected record
+	 * @param shardStateIndex index of the shard to update in subscribedShardsState;
+	 *                        this index should be the returned value from
+	 *                        {@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}, called
+	 *                        when the shard state was registered.
+	 * @param lastSequenceNumber the last sequence number value to update
+	 */
+	protected void emitRecordAndUpdateState(T record, long recordTimestamp, int shardStateIndex, SequenceNumber lastSequenceNumber) {
+		synchronized (checkpointLock) {
+			sourceContext.collectWithTimestamp(record, recordTimestamp);
+			updateState(shardStateIndex, lastSequenceNumber);
+		}
+	}
+
+	/**
+	 * Update the shard to last processed sequence number state.
+	 * This method is called by {@link ShardConsumer}s.
+	 *
+	 * @param shardStateIndex index of the shard to update in subscribedShardsState;
+	 *                        this index should be the returned value from
+	 *                        {@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}, called
+	 *                        when the shard state was registered.
+	 * @param lastSequenceNumber the last sequence number value to update
+	 */
+	protected void updateState(int shardStateIndex, SequenceNumber lastSequenceNumber) {
+		synchronized (checkpointLock) {
+			subscribedShardsState.get(shardStateIndex).setLastProcessedSequenceNum(lastSequenceNumber);
+
+			// if a shard's state is updated to be SENTINEL_SHARD_ENDING_SEQUENCE_NUM by its consumer thread,
+			// we've finished reading the shard and should determine it to be non-active
+			if (lastSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
+				this.numberOfActiveShards.decrementAndGet();
+				LOG.info("Subtask {} has reached the end of subscribed shard: {}",
+					indexOfThisConsumerSubtask, subscribedShardsState.get(shardStateIndex).getKinesisStreamShard());
+			}
+		}
+	}
+
+	/**
+	 * Register a new subscribed shard state.
+	 *
+	 * @param newSubscribedShardState the new shard state that this fetcher is to be subscribed to
+	 */
+	public int registerNewSubscribedShardState(KinesisStreamShardState newSubscribedShardState) {
+		synchronized (checkpointLock) {
+			subscribedShardsState.add(newSubscribedShardState);
+
+			// If a registered shard has initial state that is not SENTINEL_SHARD_ENDING_SEQUENCE_NUM (will be the case
+			// if the consumer had already finished reading a shard before we failed and restored), we determine that
+			// this subtask has a new active shard
+			if (!newSubscribedShardState.getLastProcessedSequenceNum().equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
+				this.numberOfActiveShards.incrementAndGet();
+			}
+
+			return subscribedShardsState.size()-1;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Miscellaneous utility functions
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Utility function to determine whether a shard should be subscribed by this consumer subtask.
+	 *
+	 * @param shard the shard to determine
+	 * @param totalNumberOfConsumerSubtasks total number of consumer subtasks
+	 * @param indexOfThisConsumerSubtask index of this consumer subtask
+	 */
+	private static boolean isThisSubtaskShouldSubscribeTo(KinesisStreamShard shard,
+														int totalNumberOfConsumerSubtasks,
+														int indexOfThisConsumerSubtask) {
+		return (Math.abs(shard.hashCode() % totalNumberOfConsumerSubtasks)) == indexOfThisConsumerSubtask;
+	}
+
+	private static ExecutorService createShardConsumersThreadPool(final String subtaskName) {
+		return Executors.newCachedThreadPool(new ThreadFactory() {
+			@Override
+			public Thread newThread(Runnable runnable) {
+				final AtomicLong threadCount = new AtomicLong(0);
+				Thread thread = new Thread(runnable);
+				thread.setName("shardConsumers-" + subtaskName + "-thread-" + threadCount.getAndIncrement());
+				thread.setDaemon(true);
+				return thread;
+			}
+		});
+	}
+
+	/**
+	 * Utility function to create an initial map of the last discovered shard id of each subscribed stream, set to null;
+	 * This is called in the constructor; correct values will be set later on by calling advanceLastDiscoveredShardOfStream()
+	 *
+	 * @param streams the list of subscribed streams
+	 * @return the initial map for subscribedStreamsToLastDiscoveredShardIds
+	 */
+	protected static HashMap<String, String> createInitialSubscribedStreamsToLastDiscoveredShardsState(List<String> streams) {
+		HashMap<String, String> initial = new HashMap<>();
+		for (String stream : streams) {
+			initial.put(stream, null);
+		}
+		return initial;
+	}
+}


[46/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
deleted file mode 100644
index b4246f5..0000000
--- a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
+++ /dev/null
@@ -1,404 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.jdbc;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.sql.Array;
-import java.sql.Connection;
-import java.sql.Date;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.Arrays;
-
-import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.io.RichInputFormat;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.api.table.typeutils.RowTypeInfo;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * InputFormat to read data from a database and generate Rows.
- * The InputFormat has to be configured using the supplied InputFormatBuilder.
- * A valid RowTypeInfo must be properly configured in the builder, e.g.: </br>
- *
- * <pre><code>
- * TypeInformation<?>[] fieldTypes = new TypeInformation<?>[] {
- *		BasicTypeInfo.INT_TYPE_INFO,
- *		BasicTypeInfo.STRING_TYPE_INFO,
- *		BasicTypeInfo.STRING_TYPE_INFO,
- *		BasicTypeInfo.DOUBLE_TYPE_INFO,
- *		BasicTypeInfo.INT_TYPE_INFO
- *	};
- *
- * RowTypeInfo rowTypeInfo = new RowTypeInfo(fieldTypes);
- *
- * JDBCInputFormat jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
- *				.setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
- *				.setDBUrl("jdbc:derby:memory:ebookshop")
- *				.setQuery("select * from books")
- *				.setRowTypeInfo(rowTypeInfo)
- *				.finish();
- * </code></pre>
- *
- * In order to query the JDBC source in parallel, you need to provide a
- * parameterized query template (i.e. a valid {@link PreparedStatement}) and
- * a {@link ParameterValuesProvider} which provides binding values for the
- * query parameters. E.g.:</br>
- *
- * <pre><code>
- *
- * Serializable[][] queryParameters = new String[2][1];
- * queryParameters[0] = new String[]{"Kumar"};
- * queryParameters[1] = new String[]{"Tan Ah Teck"};
- *
- * JDBCInputFormat jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
- *				.setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
- *				.setDBUrl("jdbc:derby:memory:ebookshop")
- *				.setQuery("select * from books WHERE author = ?")
- *				.setRowTypeInfo(rowTypeInfo)
- *				.setParametersProvider(new GenericParameterValuesProvider(queryParameters))
- *				.finish();
- * </code></pre>
- *
- * @see Row
- * @see ParameterValuesProvider
- * @see PreparedStatement
- * @see DriverManager
- */
-public class JDBCInputFormat extends RichInputFormat<Row, InputSplit> implements ResultTypeQueryable {
-
-	private static final long serialVersionUID = 1L;
-	private static final Logger LOG = LoggerFactory.getLogger(JDBCInputFormat.class);
-
-	private String username;
-	private String password;
-	private String drivername;
-	private String dbURL;
-	private String queryTemplate;
-	private int resultSetType;
-	private int resultSetConcurrency;
-	private RowTypeInfo rowTypeInfo;
-
-	private transient Connection dbConn;
-	private transient PreparedStatement statement;
-	private transient ResultSet resultSet;
-
-	private boolean hasNext;
-	private Object[][] parameterValues;
-
-	public JDBCInputFormat() {
-	}
-
-	@Override
-	public RowTypeInfo getProducedType() {
-		return rowTypeInfo;
-	}
-
-	@Override
-	public void configure(Configuration parameters) {
-		//do nothing here
-	}
-
-	@Override
-	public void openInputFormat() {
-		//called once per inputFormat (on open)
-		try {
-			Class.forName(drivername);
-			if (username == null) {
-				dbConn = DriverManager.getConnection(dbURL);
-			} else {
-				dbConn = DriverManager.getConnection(dbURL, username, password);
-			}
-			statement = dbConn.prepareStatement(queryTemplate, resultSetType, resultSetConcurrency);
-		} catch (SQLException se) {
-			throw new IllegalArgumentException("open() failed." + se.getMessage(), se);
-		} catch (ClassNotFoundException cnfe) {
-			throw new IllegalArgumentException("JDBC-Class not found. - " + cnfe.getMessage(), cnfe);
-		}
-	}
-
-	@Override
-	public void closeInputFormat() {
-		//called once per inputFormat (on close)
-		try {
-			if(statement != null) {
-				statement.close();
-			}
-		} catch (SQLException se) {
-			LOG.info("Inputformat Statement couldn't be closed - " + se.getMessage());
-		} finally {
-			statement = null;
-		}
-
-		try {
-			if(dbConn != null) {
-				dbConn.close();
-			}
-		} catch (SQLException se) {
-			LOG.info("Inputformat couldn't be closed - " + se.getMessage());
-		} finally {
-			dbConn = null;
-		}
-
-		parameterValues = null;
-	}
-
-	/**
-	 * Connects to the source database and executes the query in a <b>parallel
-	 * fashion</b> if
-	 * this {@link InputFormat} is built using a parameterized query (i.e. using
-	 * a {@link PreparedStatement})
-	 * and a proper {@link ParameterValuesProvider}, in a <b>non-parallel
-	 * fashion</b> otherwise.
-	 *
-	 * @param inputSplit which is ignored if this InputFormat is executed as a
-	 *        non-parallel source,
-	 *        a "hook" to the query parameters otherwise (using its
-	 *        <i>splitNumber</i>)
-	 * @throws IOException if there's an error during the execution of the query
-	 */
-	@Override
-	public void open(InputSplit inputSplit) throws IOException {
-		try {
-			if (inputSplit != null && parameterValues != null) {
-				for (int i = 0; i < parameterValues[inputSplit.getSplitNumber()].length; i++) {
-					Object param = parameterValues[inputSplit.getSplitNumber()][i];
-					if (param instanceof String) {
-						statement.setString(i + 1, (String) param);
-					} else if (param instanceof Long) {
-						statement.setLong(i + 1, (Long) param);
-					} else if (param instanceof Integer) {
-						statement.setInt(i + 1, (Integer) param);
-					} else if (param instanceof Double) {
-						statement.setDouble(i + 1, (Double) param);
-					} else if (param instanceof Boolean) {
-						statement.setBoolean(i + 1, (Boolean) param);
-					} else if (param instanceof Float) {
-						statement.setFloat(i + 1, (Float) param);
-					} else if (param instanceof BigDecimal) {
-						statement.setBigDecimal(i + 1, (BigDecimal) param);
-					} else if (param instanceof Byte) {
-						statement.setByte(i + 1, (Byte) param);
-					} else if (param instanceof Short) {
-						statement.setShort(i + 1, (Short) param);
-					} else if (param instanceof Date) {
-						statement.setDate(i + 1, (Date) param);
-					} else if (param instanceof Time) {
-						statement.setTime(i + 1, (Time) param);
-					} else if (param instanceof Timestamp) {
-						statement.setTimestamp(i + 1, (Timestamp) param);
-					} else if (param instanceof Array) {
-						statement.setArray(i + 1, (Array) param);
-					} else {
-						//extends with other types if needed
-						throw new IllegalArgumentException("open() failed. Parameter " + i + " of type " + param.getClass() + " is not handled (yet)." );
-					}
-				}
-				if (LOG.isDebugEnabled()) {
-					LOG.debug(String.format("Executing '%s' with parameters %s", queryTemplate, Arrays.deepToString(parameterValues[inputSplit.getSplitNumber()])));
-				}
-			}
-			resultSet = statement.executeQuery();
-			hasNext = resultSet.next();
-		} catch (SQLException se) {
-			throw new IllegalArgumentException("open() failed." + se.getMessage(), se);
-		}
-	}
-
-	/**
-	 * Closes all resources used.
-	 *
-	 * @throws IOException Indicates that a resource could not be closed.
-	 */
-	@Override
-	public void close() throws IOException {
-		if(resultSet == null) {
-			return;
-		}
-		try {
-			resultSet.close();
-		} catch (SQLException se) {
-			LOG.info("Inputformat ResultSet couldn't be closed - " + se.getMessage());
-		}
-	}
-
-	/**
-	 * Checks whether all data has been read.
-	 *
-	 * @return boolean value indication whether all data has been read.
-	 * @throws IOException
-	 */
-	@Override
-	public boolean reachedEnd() throws IOException {
-		return !hasNext;
-	}
-
-	/**
-	 * Stores the next resultSet row in a tuple
-	 *
-	 * @param row row to be reused.
-	 * @return row containing next {@link Row}
-	 * @throws java.io.IOException
-	 */
-	@Override
-	public Row nextRecord(Row row) throws IOException {
-		try {
-			if (!hasNext) {
-				return null;
-			}
-			for (int pos = 0; pos < row.productArity(); pos++) {
-				row.setField(pos, resultSet.getObject(pos + 1));
-			}
-			//update hasNext after we've read the record
-			hasNext = resultSet.next();
-			return row;
-		} catch (SQLException se) {
-			throw new IOException("Couldn't read data - " + se.getMessage(), se);
-		} catch (NullPointerException npe) {
-			throw new IOException("Couldn't access resultSet", npe);
-		}
-	}
-
-	@Override
-	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
-		return cachedStatistics;
-	}
-
-	@Override
-	public InputSplit[] createInputSplits(int minNumSplits) throws IOException {
-		if (parameterValues == null) {
-			return new GenericInputSplit[]{new GenericInputSplit(0, 1)};
-		}
-		GenericInputSplit[] ret = new GenericInputSplit[parameterValues.length];
-		for (int i = 0; i < ret.length; i++) {
-			ret[i] = new GenericInputSplit(i, ret.length);
-		}
-		return ret;
-	}
-
-	@Override
-	public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) {
-		return new DefaultInputSplitAssigner(inputSplits);
-	}
-
-	/**
-	 * A builder used to set parameters to the output format's configuration in a fluent way.
-	 * @return builder
-	 */
-	public static JDBCInputFormatBuilder buildJDBCInputFormat() {
-		return new JDBCInputFormatBuilder();
-	}
-
-	public static class JDBCInputFormatBuilder {
-		private final JDBCInputFormat format;
-
-		public JDBCInputFormatBuilder() {
-			this.format = new JDBCInputFormat();
-			//using TYPE_FORWARD_ONLY for high performance reads
-			this.format.resultSetType = ResultSet.TYPE_FORWARD_ONLY;
-			this.format.resultSetConcurrency = ResultSet.CONCUR_READ_ONLY;
-		}
-
-		public JDBCInputFormatBuilder setUsername(String username) {
-			format.username = username;
-			return this;
-		}
-
-		public JDBCInputFormatBuilder setPassword(String password) {
-			format.password = password;
-			return this;
-		}
-
-		public JDBCInputFormatBuilder setDrivername(String drivername) {
-			format.drivername = drivername;
-			return this;
-		}
-
-		public JDBCInputFormatBuilder setDBUrl(String dbURL) {
-			format.dbURL = dbURL;
-			return this;
-		}
-
-		public JDBCInputFormatBuilder setQuery(String query) {
-			format.queryTemplate = query;
-			return this;
-		}
-
-		public JDBCInputFormatBuilder setResultSetType(int resultSetType) {
-			format.resultSetType = resultSetType;
-			return this;
-		}
-
-		public JDBCInputFormatBuilder setResultSetConcurrency(int resultSetConcurrency) {
-			format.resultSetConcurrency = resultSetConcurrency;
-			return this;
-		}
-
-		public JDBCInputFormatBuilder setParametersProvider(ParameterValuesProvider parameterValuesProvider) {
-			format.parameterValues = parameterValuesProvider.getParameterValues();
-			return this;
-		}
-
-		public JDBCInputFormatBuilder setRowTypeInfo(RowTypeInfo rowTypeInfo) {
-			format.rowTypeInfo = rowTypeInfo;
-			return this;
-		}
-
-		public JDBCInputFormat finish() {
-			if (format.username == null) {
-				LOG.info("Username was not supplied separately.");
-			}
-			if (format.password == null) {
-				LOG.info("Password was not supplied separately.");
-			}
-			if (format.dbURL == null) {
-				throw new IllegalArgumentException("No database URL supplied");
-			}
-			if (format.queryTemplate == null) {
-				throw new IllegalArgumentException("No query supplied");
-			}
-			if (format.drivername == null) {
-				throw new IllegalArgumentException("No driver supplied");
-			}
-			if (format.rowTypeInfo == null) {
-				throw new IllegalArgumentException("No " + RowTypeInfo.class.getSimpleName() + " supplied");
-			}
-			if (format.parameterValues == null) {
-				LOG.debug("No input splitting configured (data will be read with parallelism 1).");
-			}
-			return format;
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java
deleted file mode 100644
index da4b1ad..0000000
--- a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java
+++ /dev/null
@@ -1,315 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.io.jdbc;
-
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-
-import org.apache.flink.api.common.io.RichOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.configuration.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * OutputFormat to write tuples into a database.
- * The OutputFormat has to be configured using the supplied OutputFormatBuilder.
- * 
- * @see Tuple
- * @see DriverManager
- */
-public class JDBCOutputFormat extends RichOutputFormat<Row> {
-	private static final long serialVersionUID = 1L;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(JDBCOutputFormat.class);
-	
-	private String username;
-	private String password;
-	private String drivername;
-	private String dbURL;
-	private String query;
-	private int batchInterval = 5000;
-	
-	private Connection dbConn;
-	private PreparedStatement upload;
-	
-	private int batchCount = 0;
-	
-	public int[] typesArray;
-	
-	public JDBCOutputFormat() {
-	}
-	
-	@Override
-	public void configure(Configuration parameters) {
-	}
-	
-	/**
-	 * Connects to the target database and initializes the prepared statement.
-	 *
-	 * @param taskNumber The number of the parallel instance.
-	 * @throws IOException Thrown, if the output could not be opened due to an
-	 * I/O problem.
-	 */
-	@Override
-	public void open(int taskNumber, int numTasks) throws IOException {
-		try {
-			establishConnection();
-			upload = dbConn.prepareStatement(query);
-		} catch (SQLException sqe) {
-			throw new IllegalArgumentException("open() failed.", sqe);
-		} catch (ClassNotFoundException cnfe) {
-			throw new IllegalArgumentException("JDBC driver class not found.", cnfe);
-		}
-	}
-	
-	private void establishConnection() throws SQLException, ClassNotFoundException {
-		Class.forName(drivername);
-		if (username == null) {
-			dbConn = DriverManager.getConnection(dbURL);
-		} else {
-			dbConn = DriverManager.getConnection(dbURL, username, password);
-		}
-	}
-	
-	/**
-	 * Adds a record to the prepared statement.
-	 * <p>
-	 * When this method is called, the output format is guaranteed to be opened.
-	 * </p>
-	 * 
-	 * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to
-	 * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null))
-	 *
-	 * @param row The records to add to the output.
-	 * @see PreparedStatement
-	 * @throws IOException Thrown, if the records could not be added due to an I/O problem.
-	 */
-	@Override
-	public void writeRecord(Row row) throws IOException {
-
-		if (typesArray != null && typesArray.length > 0 && typesArray.length != row.productArity()) {
-			LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array...");
-		} 
-		try {
-
-			if (typesArray == null ) {
-				// no types provided
-				for (int index = 0; index < row.productArity(); index++) {
-					LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.productElement(index));
-					upload.setObject(index + 1, row.productElement(index));
-				}
-			} else {
-				// types provided
-				for (int index = 0; index < row.productArity(); index++) {
-
-					if (row.productElement(index) == null) {
-						upload.setNull(index + 1, typesArray[index]);
-					} else {
-						// casting values as suggested by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html
-						switch (typesArray[index]) {
-							case java.sql.Types.NULL:
-								upload.setNull(index + 1, typesArray[index]);
-								break;
-							case java.sql.Types.BOOLEAN:
-							case java.sql.Types.BIT:
-								upload.setBoolean(index + 1, (boolean) row.productElement(index));
-								break;
-							case java.sql.Types.CHAR:
-							case java.sql.Types.NCHAR:
-							case java.sql.Types.VARCHAR:
-							case java.sql.Types.LONGVARCHAR:
-							case java.sql.Types.LONGNVARCHAR:
-								upload.setString(index + 1, (String) row.productElement(index));
-								break;
-							case java.sql.Types.TINYINT:
-								upload.setByte(index + 1, (byte) row.productElement(index));
-								break;
-							case java.sql.Types.SMALLINT:
-								upload.setShort(index + 1, (short) row.productElement(index));
-								break;
-							case java.sql.Types.INTEGER:
-								upload.setInt(index + 1, (int) row.productElement(index));
-								break;
-							case java.sql.Types.BIGINT:
-								upload.setLong(index + 1, (long) row.productElement(index));
-								break;
-							case java.sql.Types.REAL:
-								upload.setFloat(index + 1, (float) row.productElement(index));
-								break;
-							case java.sql.Types.FLOAT:
-							case java.sql.Types.DOUBLE:
-								upload.setDouble(index + 1, (double) row.productElement(index));
-								break;
-							case java.sql.Types.DECIMAL:
-							case java.sql.Types.NUMERIC:
-								upload.setBigDecimal(index + 1, (java.math.BigDecimal) row.productElement(index));
-								break;
-							case java.sql.Types.DATE:
-								upload.setDate(index + 1, (java.sql.Date) row.productElement(index));
-								break;
-							case java.sql.Types.TIME:
-								upload.setTime(index + 1, (java.sql.Time) row.productElement(index));
-								break;
-							case java.sql.Types.TIMESTAMP:
-								upload.setTimestamp(index + 1, (java.sql.Timestamp) row.productElement(index));
-								break;
-							case java.sql.Types.BINARY:
-							case java.sql.Types.VARBINARY:
-							case java.sql.Types.LONGVARBINARY:
-								upload.setBytes(index + 1, (byte[]) row.productElement(index));
-								break;
-							default:
-								upload.setObject(index + 1, row.productElement(index));
-								LOG.warn("Unmanaged sql type (%s) for column %s. Best effort approach to set its value: %s.",
-									typesArray[index], index + 1, row.productElement(index));
-								// case java.sql.Types.SQLXML
-								// case java.sql.Types.ARRAY:
-								// case java.sql.Types.JAVA_OBJECT:
-								// case java.sql.Types.BLOB:
-								// case java.sql.Types.CLOB:
-								// case java.sql.Types.NCLOB:
-								// case java.sql.Types.DATALINK:
-								// case java.sql.Types.DISTINCT:
-								// case java.sql.Types.OTHER:
-								// case java.sql.Types.REF:
-								// case java.sql.Types.ROWID:
-								// case java.sql.Types.STRUC
-						}
-					}
-				}
-			}
-			upload.addBatch();
-			batchCount++;
-			if (batchCount >= batchInterval) {
-				upload.executeBatch();
-				batchCount = 0;
-			}
-		} catch (SQLException | IllegalArgumentException e) {
-			throw new IllegalArgumentException("writeRecord() failed", e);
-		}
-	}
-	
-	/**
-	 * Executes prepared statement and closes all resources of this instance.
-	 *
-	 * @throws IOException Thrown, if the input could not be closed properly.
-	 */
-	@Override
-	public void close() throws IOException {
-		try {
-			if (upload != null) {
-				upload.executeBatch();
-				upload.close();
-			}
-		} catch (SQLException se) {
-			LOG.info("Inputformat couldn't be closed - " + se.getMessage());
-		} finally {
-			upload = null;
-			batchCount = 0;
-		}
-		
-		try {
-			if (dbConn != null) {
-				dbConn.close();
-			}
-		} catch (SQLException se) {
-			LOG.info("Inputformat couldn't be closed - " + se.getMessage());
-		} finally {
-			dbConn = null;
-		}
-	}
-	
-	public static JDBCOutputFormatBuilder buildJDBCOutputFormat() {
-		return new JDBCOutputFormatBuilder();
-	}
-	
-	public static class JDBCOutputFormatBuilder {
-		private final JDBCOutputFormat format;
-		
-		protected JDBCOutputFormatBuilder() {
-			this.format = new JDBCOutputFormat();
-		}
-		
-		public JDBCOutputFormatBuilder setUsername(String username) {
-			format.username = username;
-			return this;
-		}
-		
-		public JDBCOutputFormatBuilder setPassword(String password) {
-			format.password = password;
-			return this;
-		}
-		
-		public JDBCOutputFormatBuilder setDrivername(String drivername) {
-			format.drivername = drivername;
-			return this;
-		}
-		
-		public JDBCOutputFormatBuilder setDBUrl(String dbURL) {
-			format.dbURL = dbURL;
-			return this;
-		}
-		
-		public JDBCOutputFormatBuilder setQuery(String query) {
-			format.query = query;
-			return this;
-		}
-		
-		public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) {
-			format.batchInterval = batchInterval;
-			return this;
-		}
-		
-		public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) {
-			format.typesArray = typesArray;
-			return this;
-		}
-		
-		/**
-		 * Finalizes the configuration and checks validity.
-		 * 
-		 * @return Configured JDBCOutputFormat
-		 */
-		public JDBCOutputFormat finish() {
-			if (format.username == null) {
-				LOG.info("Username was not supplied separately.");
-			}
-			if (format.password == null) {
-				LOG.info("Password was not supplied separately.");
-			}
-			if (format.dbURL == null) {
-				throw new IllegalArgumentException("No dababase URL supplied.");
-			}
-			if (format.query == null) {
-				throw new IllegalArgumentException("No query suplied");
-			}
-			if (format.drivername == null) {
-				throw new IllegalArgumentException("No driver supplied");
-			}
-			
-			return format;
-		}
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java
deleted file mode 100644
index 2ed2f8c..0000000
--- a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.api.java.io.jdbc.split;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.java.io.jdbc.JDBCInputFormat;
-
-/** 
- * 
- * This splits generator actually does nothing but wrapping the query parameters
- * computed by the user before creating the {@link JDBCInputFormat} instance.
- * 
- * */
-public class GenericParameterValuesProvider implements ParameterValuesProvider {
-
-	private final Serializable[][] parameters;
-	
-	public GenericParameterValuesProvider(Serializable[][] parameters) {
-		this.parameters = parameters;
-	}
-
-	@Override
-	public Serializable[][] getParameterValues(){
-		//do nothing...precomputed externally
-		return parameters;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java
deleted file mode 100644
index ac56b98..0000000
--- a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.api.java.io.jdbc.split;
-
-import java.io.Serializable;
-
-/** 
- * 
- * This query generator assumes that the query to parameterize contains a BETWEEN constraint on a numeric column.
- * The generated query set will be of size equal to the configured fetchSize (apart the last one range),
- * ranging from the min value up to the max.
- * 
- * For example, if there's a table <CODE>BOOKS</CODE> with a numeric PK <CODE>id</CODE>, using a query like:
- * <PRE>
- *   SELECT * FROM BOOKS WHERE id BETWEEN ? AND ?
- * </PRE>
- *
- * you can use this class to automatically generate the parameters of the BETWEEN clause,
- * based on the passed constructor parameters.
- * 
- * */
-public class NumericBetweenParametersProvider implements ParameterValuesProvider {
-
-	private long fetchSize;
-	private final long min;
-	private final long max;
-	
-	public NumericBetweenParametersProvider(long fetchSize, long min, long max) {
-		this.fetchSize = fetchSize;
-		this.min = min;
-		this.max = max;
-	}
-
-	@Override
-	public Serializable[][] getParameterValues(){
-		double maxElemCount = (max - min) + 1;
-		int size = new Double(Math.ceil(maxElemCount / fetchSize)).intValue();
-		Serializable[][] parameters = new Serializable[size][2];
-		int count = 0;
-		for (long i = min; i < max; i += fetchSize, count++) {
-			long currentLimit = i + fetchSize - 1;
-			parameters[count] = new Long[]{i,currentLimit};
-			if (currentLimit + 1 + fetchSize > max) {
-				parameters[count + 1] = new Long[]{currentLimit + 1, max};
-				break;
-			}
-		}
-		return parameters;
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java
deleted file mode 100644
index c194497..0000000
--- a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.api.java.io.jdbc.split;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.java.io.jdbc.JDBCInputFormat;
-
-/**
- * 
- * This interface is used by the {@link JDBCInputFormat} to compute the list of parallel query to run (i.e. splits).
- * Each query will be parameterized using a row of the matrix provided by each {@link ParameterValuesProvider} implementation
- * 
- * */
-public interface ParameterValuesProvider {
-
-	/** Returns the necessary parameters array to use for query in parallel a table */
-	public Serializable[][] getParameterValues();
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java
deleted file mode 100644
index da9469b..0000000
--- a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.io.jdbc;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Types;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.jdbc.JDBCInputFormat.JDBCInputFormatBuilder;
-import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider;
-import org.apache.flink.api.table.Row;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class JDBCFullTest extends JDBCTestBase {
-
-	@Test
-	public void testJdbcInOut() throws Exception {
-		//run without parallelism
-		runTest(false);
-
-		//cleanup
-		JDBCTestBase.tearDownClass();
-		JDBCTestBase.prepareTestDb();
-		
-		//run expliting parallelism
-		runTest(true);
-		
-	}
-
-	private void runTest(boolean exploitParallelism) {
-		ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment();
-		JDBCInputFormatBuilder inputBuilder = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername(JDBCTestBase.DRIVER_CLASS)
-				.setDBUrl(JDBCTestBase.DB_URL)
-				.setQuery(JDBCTestBase.SELECT_ALL_BOOKS)
-				.setRowTypeInfo(rowTypeInfo);
-
-		if(exploitParallelism) {
-			final int fetchSize = 1;
-			final Long min = new Long(JDBCTestBase.testData[0][0].toString());
-			final Long max = new Long(JDBCTestBase.testData[JDBCTestBase.testData.length - fetchSize][0].toString());
-			//use a "splittable" query to exploit parallelism
-			inputBuilder = inputBuilder
-					.setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_ID)
-					.setParametersProvider(new NumericBetweenParametersProvider(fetchSize, min, max));
-		}
-		DataSet<Row> source = environment.createInput(inputBuilder.finish());
-
-		//NOTE: in this case (with Derby driver) setSqlTypes could be skipped, but
-		//some database, doens't handle correctly null values when no column type specified
-		//in PreparedStatement.setObject (see its javadoc for more details)
-		source.output(JDBCOutputFormat.buildJDBCOutputFormat()
-				.setDrivername(JDBCTestBase.DRIVER_CLASS)
-				.setDBUrl(JDBCTestBase.DB_URL)
-				.setQuery("insert into newbooks (id,title,author,price,qty) values (?,?,?,?,?)")
-				.setSqlTypes(new int[]{Types.INTEGER, Types.VARCHAR, Types.VARCHAR,Types.DOUBLE,Types.INTEGER})
-				.finish());
-		try {
-			environment.execute();
-		} catch (Exception e) {
-			Assert.fail("JDBC full test failed. " + e.getMessage());
-		}
-
-		try (
-			Connection dbConn = DriverManager.getConnection(JDBCTestBase.DB_URL);
-			PreparedStatement statement = dbConn.prepareStatement(JDBCTestBase.SELECT_ALL_NEWBOOKS);
-			ResultSet resultSet = statement.executeQuery()
-		) {
-			int count = 0;
-			while (resultSet.next()) {
-				count++;
-			}
-			Assert.assertEquals(JDBCTestBase.testData.length, count);
-		} catch (SQLException e) {
-			Assert.fail("JDBC full test failed. " + e.getMessage());
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
deleted file mode 100644
index efae076..0000000
--- a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.io.jdbc;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.sql.ResultSet;
-
-import org.apache.flink.api.java.io.jdbc.split.GenericParameterValuesProvider;
-import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider;
-import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider;
-import org.apache.flink.api.table.Row;
-import org.apache.flink.core.io.InputSplit;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class JDBCInputFormatTest extends JDBCTestBase {
-
-	private JDBCInputFormat jdbcInputFormat;
-
-	@After
-	public void tearDown() throws IOException {
-		if (jdbcInputFormat != null) {
-			jdbcInputFormat.close();
-		}
-		jdbcInputFormat = null;
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testUntypedRowInfo() throws IOException {
-		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername("org.apache.derby.jdbc.idontexist")
-				.setDBUrl(DB_URL)
-				.setQuery(SELECT_ALL_BOOKS)
-				.finish();
-		jdbcInputFormat.openInputFormat();
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testInvalidDriver() throws IOException {
-		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername("org.apache.derby.jdbc.idontexist")
-				.setDBUrl(DB_URL)
-				.setQuery(SELECT_ALL_BOOKS)
-				.setRowTypeInfo(rowTypeInfo)
-				.finish();
-		jdbcInputFormat.openInputFormat();
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testInvalidURL() throws IOException {
-		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl("jdbc:der:iamanerror:mory:ebookshop")
-				.setQuery(SELECT_ALL_BOOKS)
-				.setRowTypeInfo(rowTypeInfo)
-				.finish();
-		jdbcInputFormat.openInputFormat();
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testInvalidQuery() throws IOException {
-		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl(DB_URL)
-				.setQuery("iamnotsql")
-				.setRowTypeInfo(rowTypeInfo)
-				.finish();
-		jdbcInputFormat.openInputFormat();
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testIncompleteConfiguration() throws IOException {
-		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setQuery(SELECT_ALL_BOOKS)
-				.setRowTypeInfo(rowTypeInfo)
-				.finish();
-	}
-
-	@Test
-	public void testJDBCInputFormatWithoutParallelism() throws IOException, InstantiationException, IllegalAccessException {
-		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl(DB_URL)
-				.setQuery(SELECT_ALL_BOOKS)
-				.setRowTypeInfo(rowTypeInfo)
-				.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
-				.finish();
-		//this query does not exploit parallelism
-		Assert.assertEquals(1, jdbcInputFormat.createInputSplits(1).length);
-		jdbcInputFormat.openInputFormat();
-		jdbcInputFormat.open(null);
-		Row row =  new Row(5);
-		int recordCount = 0;
-		while (!jdbcInputFormat.reachedEnd()) {
-			Row next = jdbcInputFormat.nextRecord(row);
-			if (next == null) {
-				break;
-			}
-			
-			if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());}
-			if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());}
-			if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());}
-			if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());}
-			if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());}
-
-			for (int x = 0; x < 5; x++) {
-				if(testData[recordCount][x]!=null) {
-					Assert.assertEquals(testData[recordCount][x], next.productElement(x));
-				}
-			}
-			recordCount++;
-		}
-		jdbcInputFormat.close();
-		jdbcInputFormat.closeInputFormat();
-		Assert.assertEquals(testData.length, recordCount);
-	}
-	
-	@Test
-	public void testJDBCInputFormatWithParallelismAndNumericColumnSplitting() throws IOException, InstantiationException, IllegalAccessException {
-		final int fetchSize = 1;
-		final Long min = new Long(JDBCTestBase.testData[0][0] + "");
-		final Long max = new Long(JDBCTestBase.testData[JDBCTestBase.testData.length - fetchSize][0] + "");
-		ParameterValuesProvider pramProvider = new NumericBetweenParametersProvider(fetchSize, min, max);
-		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl(DB_URL)
-				.setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_ID)
-				.setRowTypeInfo(rowTypeInfo)
-				.setParametersProvider(pramProvider)
-				.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
-				.finish();
-
-		jdbcInputFormat.openInputFormat();
-		InputSplit[] splits = jdbcInputFormat.createInputSplits(1);
-		//this query exploit parallelism (1 split for every id)
-		Assert.assertEquals(testData.length, splits.length);
-		int recordCount = 0;
-		Row row =  new Row(5);
-		for (int i = 0; i < splits.length; i++) {
-			jdbcInputFormat.open(splits[i]);
-			while (!jdbcInputFormat.reachedEnd()) {
-				Row next = jdbcInputFormat.nextRecord(row);
-				if (next == null) {
-					break;
-				}
-				if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());}
-				if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());}
-				if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());}
-				if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());}
-				if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());}
-
-				for (int x = 0; x < 5; x++) {
-					if(testData[recordCount][x]!=null) {
-						Assert.assertEquals(testData[recordCount][x], next.productElement(x));
-					}
-				}
-				recordCount++;
-			}
-			jdbcInputFormat.close();
-		}
-		jdbcInputFormat.closeInputFormat();
-		Assert.assertEquals(testData.length, recordCount);
-	}
-	
-	@Test
-	public void testJDBCInputFormatWithParallelismAndGenericSplitting() throws IOException, InstantiationException, IllegalAccessException {
-		Serializable[][] queryParameters = new String[2][1];
-		queryParameters[0] = new String[]{"Kumar"};
-		queryParameters[1] = new String[]{"Tan Ah Teck"};
-		ParameterValuesProvider paramProvider = new GenericParameterValuesProvider(queryParameters);
-		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl(DB_URL)
-				.setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR)
-				.setRowTypeInfo(rowTypeInfo)
-				.setParametersProvider(paramProvider)
-				.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
-				.finish();
-		jdbcInputFormat.openInputFormat();
-		InputSplit[] splits = jdbcInputFormat.createInputSplits(1);
-		//this query exploit parallelism (1 split for every queryParameters row)
-		Assert.assertEquals(queryParameters.length, splits.length);
-		int recordCount = 0;
-		Row row =  new Row(5);
-		for (int i = 0; i < splits.length; i++) {
-			jdbcInputFormat.open(splits[i]);
-			while (!jdbcInputFormat.reachedEnd()) {
-				Row next = jdbcInputFormat.nextRecord(row);
-				if (next == null) {
-					break;
-				}
-				if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());}
-				if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());}
-				if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());}
-				if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());}
-				if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());}
-
-				recordCount++;
-			}
-			jdbcInputFormat.close();
-		}
-		Assert.assertEquals(3, recordCount);
-		jdbcInputFormat.closeInputFormat();
-	}
-	
-	@Test
-	public void testEmptyResults() throws IOException, InstantiationException, IllegalAccessException {
-		jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl(DB_URL)
-				.setQuery(SELECT_EMPTY)
-				.setRowTypeInfo(rowTypeInfo)
-				.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
-				.finish();
-		jdbcInputFormat.openInputFormat();
-		jdbcInputFormat.open(null);
-		Row row = new Row(5);
-		int recordsCnt = 0;
-		while (!jdbcInputFormat.reachedEnd()) {
-			Assert.assertNull(jdbcInputFormat.nextRecord(row));
-			recordsCnt++;
-		}
-		jdbcInputFormat.close();
-		jdbcInputFormat.closeInputFormat();
-		Assert.assertEquals(0, recordsCnt);
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java
deleted file mode 100644
index 086a84c..0000000
--- a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.io.jdbc;
-
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.table.Row;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class JDBCOutputFormatTest extends JDBCTestBase {
-
-	private JDBCOutputFormat jdbcOutputFormat;
-	private Tuple5<Integer, String, String, Double, String> tuple5 = new Tuple5<>();
-
-	@After
-	public void tearDown() throws IOException {
-		if (jdbcOutputFormat != null) {
-			jdbcOutputFormat.close();
-		}
-		jdbcOutputFormat = null;
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testInvalidDriver() throws IOException {
-		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
-				.setDrivername("org.apache.derby.jdbc.idontexist")
-				.setDBUrl(DB_URL)
-				.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
-				.finish();
-		jdbcOutputFormat.open(0, 1);
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testInvalidURL() throws IOException {
-		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl("jdbc:der:iamanerror:mory:ebookshop")
-				.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
-				.finish();
-		jdbcOutputFormat.open(0, 1);
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testInvalidQuery() throws IOException {
-		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl(DB_URL)
-				.setQuery("iamnotsql")
-				.finish();
-		jdbcOutputFormat.open(0, 1);
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testIncompleteConfiguration() throws IOException {
-		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
-				.finish();
-	}
-
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testIncompatibleTypes() throws IOException {
-		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl(DB_URL)
-				.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
-				.finish();
-		jdbcOutputFormat.open(0, 1);
-
-		tuple5.setField(4, 0);
-		tuple5.setField("hello", 1);
-		tuple5.setField("world", 2);
-		tuple5.setField(0.99, 3);
-		tuple5.setField("imthewrongtype", 4);
-
-		Row row = new Row(tuple5.getArity());
-		for (int i = 0; i < tuple5.getArity(); i++) {
-			row.setField(i, tuple5.getField(i));
-		}
-		jdbcOutputFormat.writeRecord(row);
-		jdbcOutputFormat.close();
-	}
-
-	@Test
-	public void testJDBCOutputFormat() throws IOException, InstantiationException, IllegalAccessException {
-
-		jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat()
-				.setDrivername(DRIVER_CLASS)
-				.setDBUrl(DB_URL)
-				.setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE))
-				.finish();
-		jdbcOutputFormat.open(0, 1);
-
-		for (int i = 0; i < testData.length; i++) {
-			Row row = new Row(testData[i].length);
-			for (int j = 0; j < testData[i].length; j++) {
-				row.setField(j, testData[i][j]);
-			}
-			jdbcOutputFormat.writeRecord(row);
-		}
-
-		jdbcOutputFormat.close();
-
-		try (
-			Connection dbConn = DriverManager.getConnection(JDBCTestBase.DB_URL);
-			PreparedStatement statement = dbConn.prepareStatement(JDBCTestBase.SELECT_ALL_NEWBOOKS);
-			ResultSet resultSet = statement.executeQuery()
-		) {
-			int recordCount = 0;
-			while (resultSet.next()) {
-				Row row = new Row(tuple5.getArity());
-				for (int i = 0; i < tuple5.getArity(); i++) {
-					row.setField(i, resultSet.getObject(i + 1));
-				}
-				if (row.productElement(0) != null) {
-					Assert.assertEquals("Field 0 should be int", Integer.class, row.productElement(0).getClass());
-				}
-				if (row.productElement(1) != null) {
-					Assert.assertEquals("Field 1 should be String", String.class, row.productElement(1).getClass());
-				}
-				if (row.productElement(2) != null) {
-					Assert.assertEquals("Field 2 should be String", String.class, row.productElement(2).getClass());
-				}
-				if (row.productElement(3) != null) {
-					Assert.assertEquals("Field 3 should be float", Double.class, row.productElement(3).getClass());
-				}
-				if (row.productElement(4) != null) {
-					Assert.assertEquals("Field 4 should be int", Integer.class, row.productElement(4).getClass());
-				}
-
-				for (int x = 0; x < tuple5.getArity(); x++) {
-					if (JDBCTestBase.testData[recordCount][x] != null) {
-						Assert.assertEquals(JDBCTestBase.testData[recordCount][x], row.productElement(x));
-					}
-				}
-
-				recordCount++;
-			}
-			Assert.assertEquals(JDBCTestBase.testData.length, recordCount);
-		} catch (SQLException e) {
-			Assert.fail("JDBC OutputFormat test failed. " + e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java
deleted file mode 100644
index 69ad693..0000000
--- a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.jdbc;
-
-import java.io.OutputStream;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.typeutils.RowTypeInfo;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-
-/**
- * Base test class for JDBC Input and Output formats
- */
-public class JDBCTestBase {
-	
-	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
-	public static final String DB_URL = "jdbc:derby:memory:ebookshop";
-	public static final String INPUT_TABLE = "books";
-	public static final String OUTPUT_TABLE = "newbooks";
-	public static final String SELECT_ALL_BOOKS = "select * from " + INPUT_TABLE;
-	public static final String SELECT_ALL_NEWBOOKS = "select * from " + OUTPUT_TABLE;
-	public static final String SELECT_EMPTY = "select * from books WHERE QTY < 0";
-	public static final String INSERT_TEMPLATE = "insert into %s (id, title, author, price, qty) values (?,?,?,?,?)";
-	public static final String SELECT_ALL_BOOKS_SPLIT_BY_ID = JDBCTestBase.SELECT_ALL_BOOKS + " WHERE id BETWEEN ? AND ?";
-	public static final String SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR = JDBCTestBase.SELECT_ALL_BOOKS + " WHERE author = ?";
-	
-	protected static Connection conn;
-
-	public static final Object[][] testData = {
-			{1001, ("Java public for dummies"), ("Tan Ah Teck"), 11.11, 11},
-			{1002, ("More Java for dummies"), ("Tan Ah Teck"), 22.22, 22},
-			{1003, ("More Java for more dummies"), ("Mohammad Ali"), 33.33, 33},
-			{1004, ("A Cup of Java"), ("Kumar"), 44.44, 44},
-			{1005, ("A Teaspoon of Java"), ("Kevin Jones"), 55.55, 55},
-			{1006, ("A Teaspoon of Java 1.4"), ("Kevin Jones"), 66.66, 66},
-			{1007, ("A Teaspoon of Java 1.5"), ("Kevin Jones"), 77.77, 77},
-			{1008, ("A Teaspoon of Java 1.6"), ("Kevin Jones"), 88.88, 88},
-			{1009, ("A Teaspoon of Java 1.7"), ("Kevin Jones"), 99.99, 99},
-			{1010, ("A Teaspoon of Java 1.8"), ("Kevin Jones"), null, 1010}};
-
-	public static final TypeInformation<?>[] fieldTypes = new TypeInformation<?>[] {
-		BasicTypeInfo.INT_TYPE_INFO,
-		BasicTypeInfo.STRING_TYPE_INFO,
-		BasicTypeInfo.STRING_TYPE_INFO,
-		BasicTypeInfo.DOUBLE_TYPE_INFO,
-		BasicTypeInfo.INT_TYPE_INFO
-	};
-	
-	public static final RowTypeInfo rowTypeInfo = new RowTypeInfo(fieldTypes);
-
-	public static String getCreateQuery(String tableName) {
-		StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE ");
-		sqlQueryBuilder.append(tableName).append(" (");
-		sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,");
-		sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,");
-		sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,");
-		sqlQueryBuilder.append("price FLOAT DEFAULT NULL,");
-		sqlQueryBuilder.append("qty INT DEFAULT NULL,");
-		sqlQueryBuilder.append("PRIMARY KEY (id))");
-		return sqlQueryBuilder.toString();
-	}
-	
-	public static String getInsertQuery() {
-		StringBuilder sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES ");
-		for (int i = 0; i < JDBCTestBase.testData.length; i++) {
-			sqlQueryBuilder.append("(")
-			.append(JDBCTestBase.testData[i][0]).append(",'")
-			.append(JDBCTestBase.testData[i][1]).append("','")
-			.append(JDBCTestBase.testData[i][2]).append("',")
-			.append(JDBCTestBase.testData[i][3]).append(",")
-			.append(JDBCTestBase.testData[i][4]).append(")");
-			if (i < JDBCTestBase.testData.length - 1) {
-				sqlQueryBuilder.append(",");
-			}
-		}
-		String insertQuery = sqlQueryBuilder.toString();
-		return insertQuery;
-	}
-	
-	public static final OutputStream DEV_NULL = new OutputStream() {
-		@Override
-		public void write(int b) {
-		}
-	};
-
-	public static void prepareTestDb() throws Exception {
-		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
-		Class.forName(DRIVER_CLASS);
-		Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
-
-		//create input table
-		Statement stat = conn.createStatement();
-		stat.executeUpdate(getCreateQuery(INPUT_TABLE));
-		stat.close();
-
-		//create output table
-		stat = conn.createStatement();
-		stat.executeUpdate(getCreateQuery(OUTPUT_TABLE));
-		stat.close();
-
-		//prepare input data
-		stat = conn.createStatement();
-		stat.execute(JDBCTestBase.getInsertQuery());
-		stat.close();
-
-		conn.close();
-	}
-
-	@BeforeClass
-	public static void setUpClass() throws SQLException {
-		try {
-			System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
-			prepareDerbyDatabase();
-		} catch (ClassNotFoundException e) {
-			e.printStackTrace();
-			Assert.fail();
-		}
-	}
-
-	private static void prepareDerbyDatabase() throws ClassNotFoundException, SQLException {
-		Class.forName(DRIVER_CLASS);
-		conn = DriverManager.getConnection(DB_URL + ";create=true");
-		createTable(INPUT_TABLE);
-		createTable(OUTPUT_TABLE);
-		insertDataIntoInputTable();
-		conn.close();
-	}
-	
-	private static void createTable(String tableName) throws SQLException {
-		Statement stat = conn.createStatement();
-		stat.executeUpdate(getCreateQuery(tableName));
-		stat.close();
-	}
-	
-	private static void insertDataIntoInputTable() throws SQLException {
-		Statement stat = conn.createStatement();
-		stat.execute(JDBCTestBase.getInsertQuery());
-		stat.close();
-	}
-
-	@AfterClass
-	public static void tearDownClass() {
-		cleanUpDerbyDatabases();
-	}
-
-	private static void cleanUpDerbyDatabases() {
-		try {
-			Class.forName(DRIVER_CLASS);
-			conn = DriverManager.getConnection(DB_URL + ";create=true");
-			Statement stat = conn.createStatement();
-			stat.executeUpdate("DROP TABLE "+INPUT_TABLE);
-			stat.executeUpdate("DROP TABLE "+OUTPUT_TABLE);
-			stat.close();
-			conn.close();
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail();
-		}
-	}
-
-}

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/pom.xml b/flink-connectors/flink-avro/pom.xml
new file mode 100644
index 0000000..cdd7c78
--- /dev/null
+++ b/flink-connectors/flink-avro/pom.xml
@@ -0,0 +1,216 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+	
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.2-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-avro_2.10</artifactId>
+	<name>flink-avro</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+
+		<!-- core dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.avro</groupId>
+			<artifactId>avro</artifactId>
+			<!-- version is derived from base module -->
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>create-test-dependency</id>
+						<phase>process-test-classes</phase>
+						<goals>
+							<goal>single</goal>
+						</goals>
+						<configuration>
+							<archive>
+								<manifest>
+									<mainClass>org.apache.flink.api.avro.testjar.AvroExternalJarProgram</mainClass>
+								</manifest>
+							</archive>
+							<finalName>maven</finalName>
+							<attach>false</attach>
+							<descriptors>
+								<descriptor>src/test/assembly/test-assembly.xml</descriptor>
+							</descriptors>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<!--Remove the AvroExternalJarProgram code from the test-classes directory since it musn't be in the
+			classpath when running the tests to actually test whether the user code class loader
+			is properly used.-->
+			<plugin>
+				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version><!--$NO-MVN-MAN-VER$-->
+				<executions>
+					<execution>
+						<id>remove-avroexternalprogram</id>
+						<phase>process-test-classes</phase>
+						<goals>
+							<goal>clean</goal>
+						</goals>
+						<configuration>
+							<excludeDefaultDirectories>true</excludeDefaultDirectories>
+							<filesets>
+								<fileset>
+									<directory>${project.build.testOutputDirectory}</directory>
+									<includes>
+										<include>**/testjar/*.class</include>
+									</includes>
+								</fileset>
+							</filesets>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<!-- Generate Test class from avro schema -->
+			<plugin>
+				<groupId>org.apache.avro</groupId>
+				<artifactId>avro-maven-plugin</artifactId>
+				<version>1.7.7</version>
+				<executions>
+					<execution>
+						<phase>generate-sources</phase>
+						<goals>
+							<goal>schema</goal>
+						</goals>
+						<configuration>
+							<testSourceDirectory>${project.basedir}/src/test/resources/avro</testSourceDirectory>
+							<testOutputDirectory>${project.basedir}/src/test/java/</testOutputDirectory>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+		
+		<pluginManagement>
+			<plugins>
+				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+				<plugin>
+					<groupId>org.eclipse.m2e</groupId>
+					<artifactId>lifecycle-mapping</artifactId>
+					<version>1.0.0</version>
+					<configuration>
+						<lifecycleMappingMetadata>
+							<pluginExecutions>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.maven.plugins</groupId>
+										<artifactId>maven-assembly-plugin</artifactId>
+										<versionRange>[2.4,)</versionRange>
+										<goals>
+											<goal>single</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore/>
+									</action>
+								</pluginExecution>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.maven.plugins</groupId>
+										<artifactId>maven-clean-plugin</artifactId>
+										<versionRange>[1,)</versionRange>
+										<goals>
+											<goal>clean</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore/>
+									</action>
+								</pluginExecution>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.avro</groupId>
+										<artifactId>avro-maven-plugin</artifactId>
+										<versionRange>[1.7.7,)</versionRange>
+										<goals>
+											<goal>schema</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore/>
+									</action>
+								</pluginExecution>
+							</pluginExecutions>
+						</lifecycleMappingMetadata>
+					</configuration>
+				</plugin>
+			</plugins>
+		</pluginManagement>
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataInputDecoder.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataInputDecoder.java b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataInputDecoder.java
new file mode 100644
index 0000000..59da4cb
--- /dev/null
+++ b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataInputDecoder.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.
+ */
+
+package org.apache.flink.api.avro;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.avro.io.Decoder;
+import org.apache.avro.util.Utf8;
+
+
+public class DataInputDecoder extends Decoder {
+	
+	private final Utf8 stringDecoder = new Utf8();
+	
+	private DataInput in;
+	
+	public void setIn(DataInput in) {
+		this.in = in;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// primitives
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public void readNull() {}
+	
+
+	@Override
+	public boolean readBoolean() throws IOException {
+		return in.readBoolean();
+	}
+
+	@Override
+	public int readInt() throws IOException {
+		return in.readInt();
+	}
+
+	@Override
+	public long readLong() throws IOException {
+		return in.readLong();
+	}
+
+	@Override
+	public float readFloat() throws IOException {
+		return in.readFloat();
+	}
+
+	@Override
+	public double readDouble() throws IOException {
+		return in.readDouble();
+	}
+	
+	@Override
+	public int readEnum() throws IOException {
+		return readInt();
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	// bytes
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void readFixed(byte[] bytes, int start, int length) throws IOException {
+		in.readFully(bytes, start, length);
+	}
+	
+	@Override
+	public ByteBuffer readBytes(ByteBuffer old) throws IOException {
+		int length = readInt();
+		ByteBuffer result;
+		if (old != null && length <= old.capacity() && old.hasArray()) {
+			result = old;
+			result.clear();
+		} else {
+			result = ByteBuffer.allocate(length);
+		}
+		in.readFully(result.array(), result.arrayOffset() + result.position(), length);
+		result.limit(length);
+		return result;
+	}
+	
+	
+	@Override
+	public void skipFixed(int length) throws IOException {
+		skipBytes(length);
+	}
+	
+	@Override
+	public void skipBytes() throws IOException {
+		int num = readInt();
+		skipBytes(num);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	// strings
+	// --------------------------------------------------------------------------------------------
+	
+	
+	@Override
+	public Utf8 readString(Utf8 old) throws IOException {
+		int length = readInt();
+		Utf8 result = (old != null ? old : new Utf8());
+		result.setByteLength(length);
+		
+		if (length > 0) {
+			in.readFully(result.getBytes(), 0, length);
+		}
+		
+		return result;
+	}
+
+	@Override
+	public String readString() throws IOException {
+		return readString(stringDecoder).toString();
+	}
+
+	@Override
+	public void skipString() throws IOException {
+		int len = readInt();
+		skipBytes(len);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// collection types
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public long readArrayStart() throws IOException {
+		return readVarLongCount(in);
+	}
+
+	@Override
+	public long arrayNext() throws IOException {
+		return readVarLongCount(in);
+	}
+
+	@Override
+	public long skipArray() throws IOException {
+		return readVarLongCount(in);
+	}
+
+	@Override
+	public long readMapStart() throws IOException {
+		return readVarLongCount(in);
+	}
+
+	@Override
+	public long mapNext() throws IOException {
+		return readVarLongCount(in);
+	}
+
+	@Override
+	public long skipMap() throws IOException {
+		return readVarLongCount(in);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	// union
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int readIndex() throws IOException {
+		return readInt();
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	// utils
+	// --------------------------------------------------------------------------------------------
+	
+	private void skipBytes(int num) throws IOException {
+		while (num > 0) {
+			num -= in.skipBytes(num);
+		}
+	}
+	
+	public static long readVarLongCount(DataInput in) throws IOException {
+		long value = in.readUnsignedByte();
+
+		if ((value & 0x80) == 0) {
+			return value;
+		}
+		else {
+			long curr;
+			int shift = 7;
+			value = value & 0x7f;
+			while (((curr = in.readUnsignedByte()) & 0x80) != 0){
+				value |= (curr & 0x7f) << shift;
+				shift += 7;
+			}
+			value |= curr << shift;
+			return value;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataOutputEncoder.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataOutputEncoder.java b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataOutputEncoder.java
new file mode 100644
index 0000000..0102cc1
--- /dev/null
+++ b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/DataOutputEncoder.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.avro;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.avro.io.Encoder;
+import org.apache.avro.util.Utf8;
+
+
+public final class DataOutputEncoder extends Encoder implements java.io.Serializable {
+	
+	private static final long serialVersionUID = 1L;
+	
+	private DataOutput out;
+	
+	
+	public void setOut(DataOutput out) {
+		this.out = out;
+	}
+
+
+	@Override
+	public void flush() throws IOException {}
+
+	// --------------------------------------------------------------------------------------------
+	// primitives
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public void writeNull() {}
+	
+
+	@Override
+	public void writeBoolean(boolean b) throws IOException {
+		out.writeBoolean(b);
+	}
+
+	@Override
+	public void writeInt(int n) throws IOException {
+		out.writeInt(n);
+	}
+
+	@Override
+	public void writeLong(long n) throws IOException {
+		out.writeLong(n);
+	}
+
+	@Override
+	public void writeFloat(float f) throws IOException {
+		out.writeFloat(f);
+	}
+
+	@Override
+	public void writeDouble(double d) throws IOException {
+		out.writeDouble(d);
+	}
+	
+	@Override
+	public void writeEnum(int e) throws IOException {
+		out.writeInt(e);
+	}
+	
+	
+	// --------------------------------------------------------------------------------------------
+	// bytes
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void writeFixed(byte[] bytes, int start, int len) throws IOException {
+		out.write(bytes, start, len);
+	}
+	
+	@Override
+	public void writeBytes(byte[] bytes, int start, int len) throws IOException {
+		out.writeInt(len);
+		if (len > 0) {
+			out.write(bytes, start, len);
+		}
+	}
+	
+	@Override
+	public void writeBytes(ByteBuffer bytes) throws IOException {
+		int num = bytes.remaining();
+		out.writeInt(num);
+		
+		if (num > 0) {
+			writeFixed(bytes);
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	// strings
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void writeString(String str) throws IOException {
+		byte[] bytes = Utf8.getBytesFor(str);
+		writeBytes(bytes, 0, bytes.length);
+	}
+	
+	@Override
+	public void writeString(Utf8 utf8) throws IOException {
+		writeBytes(utf8.getBytes(), 0, utf8.getByteLength());
+		
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// collection types
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void writeArrayStart() {}
+
+	@Override
+	public void setItemCount(long itemCount) throws IOException {
+		if (itemCount > 0) {
+			writeVarLongCount(out, itemCount);
+		}
+	}
+
+	@Override
+	public void startItem() {}
+
+	@Override
+	public void writeArrayEnd() throws IOException {
+		// write a single byte 0, shortcut for a var-length long of 0
+		out.write(0);
+	}
+
+	@Override
+	public void writeMapStart() {}
+
+	@Override
+	public void writeMapEnd() throws IOException {
+		// write a single byte 0, shortcut for a var-length long of 0
+		out.write(0);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// union
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public void writeIndex(int unionIndex) throws IOException {
+		out.writeInt(unionIndex);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	// utils
+	// --------------------------------------------------------------------------------------------
+		
+	
+	public static void writeVarLongCount(DataOutput out, long val) throws IOException {
+		if (val < 0) {
+			throw new IOException("Illegal count (must be non-negative): " + val);
+		}
+		
+		while ((val & ~0x7FL) != 0) {
+			out.write(((int) val) | 0x80);
+			val >>>= 7;
+		}
+		out.write((int) val);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java
new file mode 100644
index 0000000..709c4f1
--- /dev/null
+++ b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.api.avro;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.avro.file.SeekableInput;
+import org.apache.flink.core.fs.FSDataInputStream;
+
+
+/**
+ * Code copy pasted from org.apache.avro.mapred.FSInput (which is Apache licensed as well)
+ * 
+ * The wrapper keeps track of the position in the data stream.
+ */
+public class FSDataInputStreamWrapper implements Closeable, SeekableInput {
+	private final FSDataInputStream stream;
+	private long pos;
+	private long len;
+
+	public FSDataInputStreamWrapper(FSDataInputStream stream, long len) {
+		this.stream = stream;
+		this.pos = 0;
+		this.len = len;
+	}
+
+	public long length() throws IOException {
+		return this.len;
+	}
+
+	public int read(byte[] b, int off, int len) throws IOException {
+		int read;
+		read = stream.read(b, off, len);
+		pos += read;
+		return read;
+	}
+
+	public void seek(long p) throws IOException {
+		stream.seek(p);
+		pos = p;
+	}
+
+	public long tell() throws IOException {
+		return pos;
+	}
+
+	public void close() throws IOException {
+		stream.close();
+	}
+}


[43/51] [abbrv] [partial] flink git commit: [FLINK-4676] [connectors] Merge batch and streaming connectors into common Maven module.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSink.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSink.java
new file mode 100644
index 0000000..a3d002e
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSink.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cassandra;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.ResultSetFuture;
+import com.datastax.driver.core.Session;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
+import org.apache.flink.streaming.runtime.operators.GenericWriteAheadSink;
+
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Sink that emits its input elements into a Cassandra database. This sink stores incoming records within a
+ * {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only commits them to cassandra
+ * if a checkpoint is completed.
+ *
+ * @param <IN> Type of the elements emitted by this sink
+ */
+public class CassandraTupleWriteAheadSink<IN extends Tuple> extends GenericWriteAheadSink<IN> {
+	private static final long serialVersionUID = 1L;
+
+	protected transient Cluster cluster;
+	protected transient Session session;
+
+	private final String insertQuery;
+	private transient PreparedStatement preparedStatement;
+
+	private ClusterBuilder builder;
+
+	private transient Object[] fields;
+
+	protected CassandraTupleWriteAheadSink(String insertQuery, TypeSerializer<IN> serializer, ClusterBuilder builder, CheckpointCommitter committer) throws Exception {
+		super(committer, serializer, UUID.randomUUID().toString().replace("-", "_"));
+		this.insertQuery = insertQuery;
+		this.builder = builder;
+		ClosureCleaner.clean(builder, true);
+	}
+
+	public void open() throws Exception {
+		super.open();
+		if (!getRuntimeContext().isCheckpointingEnabled()) {
+			throw new IllegalStateException("The write-ahead log requires checkpointing to be enabled.");
+		}
+		cluster = builder.getCluster();
+		session = cluster.connect();
+		preparedStatement = session.prepare(insertQuery);
+
+		fields = new Object[((TupleSerializer<IN>) serializer).getArity()];
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		try {
+			if (session != null) {
+				session.close();
+			}
+		} catch (Exception e) {
+			LOG.error("Error while closing session.", e);
+		}
+		try {
+			if (cluster != null) {
+				cluster.close();
+			}
+		} catch (Exception e) {
+			LOG.error("Error while closing cluster.", e);
+		}
+	}
+
+	@Override
+	protected boolean sendValues(Iterable<IN> values, long timestamp) throws Exception {
+		final AtomicInteger updatesCount = new AtomicInteger(0);
+		final AtomicInteger updatesConfirmed = new AtomicInteger(0);
+
+		final AtomicReference<Throwable> exception = new AtomicReference<>();
+
+		FutureCallback<ResultSet> callback = new FutureCallback<ResultSet>() {
+			@Override
+			public void onSuccess(ResultSet resultSet) {
+				updatesConfirmed.incrementAndGet();
+				if (updatesCount.get() > 0) { // only set if all updates have been sent
+					if (updatesCount.get() == updatesConfirmed.get()) {
+						synchronized (updatesConfirmed) {
+							updatesConfirmed.notifyAll();
+						}
+					}
+				}
+			}
+
+			@Override
+			public void onFailure(Throwable throwable) {
+				if (exception.compareAndSet(null, throwable)) {
+					LOG.error("Error while sending value.", throwable);
+					synchronized (updatesConfirmed) {
+						updatesConfirmed.notifyAll();
+					}
+				}
+			}
+		};
+
+		//set values for prepared statement
+		int updatesSent = 0;
+		for (IN value : values) {
+			for (int x = 0; x < value.getArity(); x++) {
+				fields[x] = value.getField(x);
+			}
+			//insert values and send to cassandra
+			BoundStatement s = preparedStatement.bind(fields);
+			s.setDefaultTimestamp(timestamp);
+			ResultSetFuture result = session.executeAsync(s);
+			updatesSent++;
+			if (result != null) {
+				//add callback to detect errors
+				Futures.addCallback(result, callback);
+			}
+		}
+		updatesCount.set(updatesSent);
+
+		synchronized (updatesConfirmed) {
+			while (exception.get() == null && updatesSent != updatesConfirmed.get()) {
+				updatesConfirmed.wait();
+			}
+		}
+
+		if (exception.get() != null) {
+			LOG.warn("Sending a value failed.", exception.get());
+			return false;
+		} else {
+			return true;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/ClusterBuilder.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/ClusterBuilder.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/ClusterBuilder.java
new file mode 100644
index 0000000..9fd3b4e
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/ClusterBuilder.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.cassandra;
+
+import com.datastax.driver.core.Cluster;
+
+import java.io.Serializable;
+
+/**
+ * This class is used to configure a {@link com.datastax.driver.core.Cluster} after deployment.
+ * The cluster represents the connection that will be established to Cassandra.
+ */
+public abstract class ClusterBuilder implements Serializable {
+
+	public Cluster getCluster() {
+		return buildCluster(Cluster.builder());
+	}
+
+	/**
+	 * Configures the connection to Cassandra.
+	 * The configuration is done by calling methods on the builder object
+	 * and finalizing the configuration with build().
+	 *
+	 * @param builder connection builder
+	 * @return configured connection
+	 */
+	protected abstract Cluster buildCluster(Cluster.Builder builder);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java
new file mode 100644
index 0000000..e66b8b3
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.batch.connectors.cassandra.example;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Cluster.Builder;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.batch.connectors.cassandra.CassandraInputFormat;
+import org.apache.flink.batch.connectors.cassandra.CassandraOutputFormat;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import java.util.ArrayList;
+
+/**
+ * This is an example showing the to use the Cassandra Input-/OutputFormats in the Batch API.
+ * 
+ * The example assumes that a table exists in a local cassandra database, according to the following query: 
+ * CREATE TABLE test.batches (number int, strings text, PRIMARY KEY(number, strings));
+ */
+public class BatchExample {
+	private static final String INSERT_QUERY = "INSERT INTO test.batches (number, strings) VALUES (?,?);";
+	private static final String SELECT_QUERY = "SELECT number, strings FROM test.batches;";
+
+	/*
+	 *	table script: "CREATE TABLE test.batches (number int, strings text, PRIMARY KEY(number, strings));"
+	 */
+	public static void main(String[] args) throws Exception {
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		ArrayList<Tuple2<Integer, String>> collection = new ArrayList<>(20);
+		for (int i = 0; i < 20; i++) {
+			collection.add(new Tuple2<>(i, "string " + i));
+		}
+
+		DataSet<Tuple2<Integer, String>> dataSet = env.fromCollection(collection);
+
+		dataSet.output(new CassandraOutputFormat<Tuple2<Integer, String>>(INSERT_QUERY, new ClusterBuilder() {
+			@Override
+			protected Cluster buildCluster(Builder builder) {
+				return builder.addContactPoints("127.0.0.1").build();
+			}
+		}));
+
+		env.execute("Write");
+
+		DataSet<Tuple2<Integer, String>> inputDS = env
+			.createInput(new CassandraInputFormat<Tuple2<Integer, String>>(SELECT_QUERY, new ClusterBuilder() {
+				@Override
+				protected Cluster buildCluster(Builder builder) {
+					return builder.addContactPoints("127.0.0.1").build();
+				}
+			}), TupleTypeInfo.of(new TypeHint<Tuple2<Integer, String>>() {
+			}));
+
+		inputDS.print();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
new file mode 100644
index 0000000..2bb6fd1
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cassandra;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+
+import org.apache.cassandra.service.CassandraDaemon;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.batch.connectors.cassandra.CassandraInputFormat;
+import org.apache.flink.batch.connectors.cassandra.CassandraOutputFormat;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.runtime.operators.WriteAheadSinkTestBase;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
+import org.apache.flink.test.util.TestEnvironment;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Scanner;
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+
+@SuppressWarnings("serial")
+public class CassandraConnectorITCase extends WriteAheadSinkTestBase<Tuple3<String, Integer, Integer>, CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>>> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CassandraConnectorITCase.class);
+	private static File tmpDir;
+
+	private static final boolean EMBEDDED = true;
+
+	private static EmbeddedCassandraService cassandra;
+
+	private static ClusterBuilder builder = new ClusterBuilder() {
+		@Override
+		protected Cluster buildCluster(Cluster.Builder builder) {
+			return builder
+				.addContactPoint("127.0.0.1")
+				.withQueryOptions(new QueryOptions().setConsistencyLevel(ConsistencyLevel.ONE).setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL))
+				.withoutJMXReporting()
+				.withoutMetrics().build();
+		}
+	};
+
+	private static Cluster cluster;
+	private static Session session;
+
+	private static final String CREATE_KEYSPACE_QUERY = "CREATE KEYSPACE flink WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};";
+	private static final String DROP_KEYSPACE_QUERY = "DROP KEYSPACE flink;";
+	private static final String CREATE_TABLE_QUERY = "CREATE TABLE flink.test (id text PRIMARY KEY, counter int, batch_id int);";
+	private static final String CLEAR_TABLE_QUERY = "TRUNCATE flink.test;";
+	private static final String INSERT_DATA_QUERY = "INSERT INTO flink.test (id, counter, batch_id) VALUES (?, ?, ?)";
+	private static final String SELECT_DATA_QUERY = "SELECT * FROM flink.test;";
+
+	private static final ArrayList<Tuple3<String, Integer, Integer>> collection = new ArrayList<>(20);
+
+	static {
+		for (int i = 0; i < 20; i++) {
+			collection.add(new Tuple3<>(UUID.randomUUID().toString(), i, 0));
+		}
+	}
+
+	private static class EmbeddedCassandraService {
+		CassandraDaemon cassandraDaemon;
+
+		public void start() throws IOException {
+			this.cassandraDaemon = new CassandraDaemon();
+			this.cassandraDaemon.init(null);
+			this.cassandraDaemon.start();
+		}
+
+		public void stop() {
+			this.cassandraDaemon.stop();
+		}
+	}
+
+	private static LocalFlinkMiniCluster flinkCluster;
+
+	// ------------------------------------------------------------------------
+	//  Cluster Setup (Cassandra & Flink)
+	// ------------------------------------------------------------------------
+
+	@BeforeClass
+	public static void startCassandra() throws IOException {
+
+		// check if we should run this test, current Cassandra version requires Java >= 1.8
+		org.apache.flink.core.testutils.CommonTestUtils.assumeJava8();
+
+		// generate temporary files
+		tmpDir = CommonTestUtils.createTempDirectory();
+		ClassLoader classLoader = CassandraConnectorITCase.class.getClassLoader();
+		File file = new File(classLoader.getResource("cassandra.yaml").getFile());
+		File tmp = new File(tmpDir.getAbsolutePath() + File.separator + "cassandra.yaml");
+		
+		assertTrue(tmp.createNewFile());
+
+		try (
+			BufferedWriter b = new BufferedWriter(new FileWriter(tmp));
+
+			//copy cassandra.yaml; inject absolute paths into cassandra.yaml
+			Scanner scanner = new Scanner(file);
+		) {
+			while (scanner.hasNextLine()) {
+				String line = scanner.nextLine();
+				line = line.replace("$PATH", "'" + tmp.getParentFile());
+				b.write(line + "\n");
+				b.flush();
+			}
+		}
+
+
+		// Tell cassandra where the configuration files are.
+		// Use the test configuration file.
+		System.setProperty("cassandra.config", tmp.getAbsoluteFile().toURI().toString());
+
+		if (EMBEDDED) {
+			cassandra = new EmbeddedCassandraService();
+			cassandra.start();
+		}
+
+		try {
+			Thread.sleep(1000 * 10);
+		} catch (InterruptedException e) { //give cassandra a few seconds to start up
+		}
+
+		cluster = builder.getCluster();
+		session = cluster.connect();
+
+		session.execute(CREATE_KEYSPACE_QUERY);
+		session.execute(CREATE_TABLE_QUERY);
+	}
+
+	@BeforeClass
+	public static void startFlink() throws Exception {
+		Configuration config = new Configuration();
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
+
+		flinkCluster = new LocalFlinkMiniCluster(config);
+		flinkCluster.start();
+	}
+
+	@AfterClass
+	public static void stopFlink() {
+		if (flinkCluster != null) {
+			flinkCluster.stop();
+			flinkCluster = null;
+		}
+	}
+
+	@AfterClass
+	public static void closeCassandra() {
+		if (session != null) {
+			session.executeAsync(DROP_KEYSPACE_QUERY);
+			session.close();
+		}
+
+		if (cluster != null) {
+			cluster.close();
+		}
+
+		if (cassandra != null) {
+			cassandra.stop();
+		}
+
+		if (tmpDir != null) {
+			//noinspection ResultOfMethodCallIgnored
+			tmpDir.delete();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Test preparation & cleanup
+	// ------------------------------------------------------------------------
+
+	@Before
+	public void initializeExecutionEnvironment() {
+		TestStreamEnvironment.setAsContext(flinkCluster, 4);
+		new TestEnvironment(flinkCluster, 4, false).setAsContext();
+	}
+
+	@After
+	public void deleteSchema() throws Exception {
+		session.executeAsync(CLEAR_TABLE_QUERY);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Exactly-once Tests
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>> createSink() throws Exception {
+		return new CassandraTupleWriteAheadSink<>(
+			INSERT_DATA_QUERY,
+			TypeExtractor.getForObject(new Tuple3<>("", 0, 0)).createSerializer(new ExecutionConfig()),
+			builder,
+			new CassandraCommitter(builder));
+	}
+
+	@Override
+	protected TupleTypeInfo<Tuple3<String, Integer, Integer>> createTypeInfo() {
+		return TupleTypeInfo.getBasicTupleTypeInfo(String.class, Integer.class, Integer.class);
+	}
+
+	@Override
+	protected Tuple3<String, Integer, Integer> generateValue(int counter, int checkpointID) {
+		return new Tuple3<>(UUID.randomUUID().toString(), counter, checkpointID);
+	}
+
+	@Override
+	protected void verifyResultsIdealCircumstances(
+		OneInputStreamOperatorTestHarness<Tuple3<String, Integer, Integer>, Tuple3<String, Integer, Integer>> harness,
+		CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>> sink) {
+
+		ResultSet result = session.execute(SELECT_DATA_QUERY);
+		ArrayList<Integer> list = new ArrayList<>();
+		for (int x = 1; x <= 60; x++) {
+			list.add(x);
+		}
+
+		for (Row s : result) {
+			list.remove(new Integer(s.getInt("counter")));
+		}
+		Assert.assertTrue("The following ID's were not found in the ResultSet: " + list.toString(), list.isEmpty());
+	}
+
+	@Override
+	protected void verifyResultsDataPersistenceUponMissedNotify(
+		OneInputStreamOperatorTestHarness<Tuple3<String, Integer, Integer>, Tuple3<String, Integer, Integer>> harness,
+		CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>> sink) {
+
+		ResultSet result = session.execute(SELECT_DATA_QUERY);
+		ArrayList<Integer> list = new ArrayList<>();
+		for (int x = 1; x <= 60; x++) {
+			list.add(x);
+		}
+
+		for (Row s : result) {
+			list.remove(new Integer(s.getInt("counter")));
+		}
+		Assert.assertTrue("The following ID's were not found in the ResultSet: " + list.toString(), list.isEmpty());
+	}
+
+	@Override
+	protected void verifyResultsDataDiscardingUponRestore(
+		OneInputStreamOperatorTestHarness<Tuple3<String, Integer, Integer>, Tuple3<String, Integer, Integer>> harness,
+		CassandraTupleWriteAheadSink<Tuple3<String, Integer, Integer>> sink) {
+
+		ResultSet result = session.execute(SELECT_DATA_QUERY);
+		ArrayList<Integer> list = new ArrayList<>();
+		for (int x = 1; x <= 20; x++) {
+			list.add(x);
+		}
+		for (int x = 41; x <= 60; x++) {
+			list.add(x);
+		}
+
+		for (Row s : result) {
+			list.remove(new Integer(s.getInt("counter")));
+		}
+		Assert.assertTrue("The following ID's were not found in the ResultSet: " + list.toString(), list.isEmpty());
+	}
+
+	@Test
+	public void testCassandraCommitter() throws Exception {
+		CassandraCommitter cc1 = new CassandraCommitter(builder);
+		cc1.setJobId("job");
+		cc1.setOperatorId("operator");
+
+		CassandraCommitter cc2 = new CassandraCommitter(builder);
+		cc2.setJobId("job");
+		cc2.setOperatorId("operator");
+
+		CassandraCommitter cc3 = new CassandraCommitter(builder);
+		cc3.setJobId("job");
+		cc3.setOperatorId("operator1");
+
+		cc1.createResource();
+
+		cc1.open();
+		cc2.open();
+		cc3.open();
+
+		Assert.assertFalse(cc1.isCheckpointCommitted(0, 1));
+		Assert.assertFalse(cc2.isCheckpointCommitted(1, 1));
+		Assert.assertFalse(cc3.isCheckpointCommitted(0, 1));
+
+		cc1.commitCheckpoint(0, 1);
+		Assert.assertTrue(cc1.isCheckpointCommitted(0, 1));
+		//verify that other sub-tasks aren't affected
+		Assert.assertFalse(cc2.isCheckpointCommitted(1, 1));
+		//verify that other tasks aren't affected
+		Assert.assertFalse(cc3.isCheckpointCommitted(0, 1));
+
+		Assert.assertFalse(cc1.isCheckpointCommitted(0, 2));
+
+		cc1.close();
+		cc2.close();
+		cc3.close();
+
+		cc1 = new CassandraCommitter(builder);
+		cc1.setJobId("job");
+		cc1.setOperatorId("operator");
+
+		cc1.open();
+
+		//verify that checkpoint data is not destroyed within open/close and not reliant on internally cached data
+		Assert.assertTrue(cc1.isCheckpointCommitted(0, 1));
+		Assert.assertFalse(cc1.isCheckpointCommitted(0, 2));
+
+		cc1.close();
+	}
+
+	// ------------------------------------------------------------------------
+	//  At-least-once Tests
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testCassandraTupleAtLeastOnceSink() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataStream<Tuple3<String, Integer, Integer>> source = env.fromCollection(collection);
+		source.addSink(new CassandraTupleSink<Tuple3<String, Integer, Integer>>(INSERT_DATA_QUERY, builder));
+
+		env.execute();
+
+		ResultSet rs = session.execute(SELECT_DATA_QUERY);
+		Assert.assertEquals(20, rs.all().size());
+	}
+
+	@Test
+	public void testCassandraPojoAtLeastOnceSink() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataStreamSource<Pojo> source = env
+			.addSource(new SourceFunction<Pojo>() {
+
+				private boolean running = true;
+				private volatile int cnt = 0;
+
+				@Override
+				public void run(SourceContext<Pojo> ctx) throws Exception {
+					while (running) {
+						ctx.collect(new Pojo(UUID.randomUUID().toString(), cnt, 0));
+						cnt++;
+						if (cnt == 20) {
+							cancel();
+						}
+					}
+				}
+
+				@Override
+				public void cancel() {
+					running = false;
+				}
+			});
+
+		source.addSink(new CassandraPojoSink<>(Pojo.class, builder));
+
+		env.execute();
+
+		ResultSet rs = session.execute(SELECT_DATA_QUERY);
+		Assert.assertEquals(20, rs.all().size());
+	}
+
+	@Test
+	public void testCassandraBatchFormats() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple3<String, Integer, Integer>> dataSet = env.fromCollection(collection);
+		dataSet.output(new CassandraOutputFormat<Tuple3<String, Integer, Integer>>(INSERT_DATA_QUERY, builder));
+
+		env.execute("Write data");
+
+		DataSet<Tuple3<String, Integer, Integer>> inputDS = env.createInput(
+			new CassandraInputFormat<Tuple3<String, Integer, Integer>>(SELECT_DATA_QUERY, builder),
+			TypeInformation.of(new TypeHint<Tuple3<String, Integer, Integer>>(){}));
+
+
+		long count = inputDS.count();
+		Assert.assertEquals(count, 20L);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSinkTest.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSinkTest.java
new file mode 100644
index 0000000..847d1a0
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSinkTest.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.cassandra;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSetFuture;
+import com.datastax.driver.core.Session;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.tuple.Tuple0;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Collections;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.powermock.api.mockito.PowerMockito.doAnswer;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+public class CassandraTupleWriteAheadSinkTest {
+
+	@Test(timeout=20000)
+	public void testAckLoopExitOnException() throws Exception {
+		final AtomicReference<Runnable> runnableFuture = new AtomicReference<>();
+
+		final ClusterBuilder clusterBuilder = new ClusterBuilder() {
+			private static final long serialVersionUID = 4624400760492936756L;
+
+			@Override
+			protected Cluster buildCluster(Cluster.Builder builder) {
+				try {
+					BoundStatement boundStatement = mock(BoundStatement.class);
+					when(boundStatement.setDefaultTimestamp(any(long.class))).thenReturn(boundStatement);
+
+					PreparedStatement preparedStatement = mock(PreparedStatement.class);
+					when(preparedStatement.bind(Matchers.anyVararg())).thenReturn(boundStatement);
+
+					ResultSetFuture future = mock(ResultSetFuture.class);
+					when(future.get()).thenThrow(new RuntimeException("Expected exception."));
+
+					doAnswer(new Answer<Void>() {
+						@Override
+						public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
+							synchronized (runnableFuture) {
+								runnableFuture.set((((Runnable) invocationOnMock.getArguments()[0])));
+								runnableFuture.notifyAll();
+							}
+							return null;
+						}
+					}).when(future).addListener(any(Runnable.class), any(Executor.class));
+
+					Session session = mock(Session.class);
+					when(session.prepare(anyString())).thenReturn(preparedStatement);
+					when(session.executeAsync(any(BoundStatement.class))).thenReturn(future);
+
+					Cluster cluster = mock(Cluster.class);
+					when(cluster.connect()).thenReturn(session);
+					return cluster;
+				} catch (Exception e) {
+					throw new RuntimeException(e);
+				}
+			}
+		};
+
+		// Our asynchronous executor thread
+		new Thread(new Runnable() {
+			@Override
+			public void run() {
+				synchronized (runnableFuture) {
+					while (runnableFuture.get() == null) {
+						try {
+							runnableFuture.wait();
+						} catch (InterruptedException e) {
+							// ignore interrupts
+						}
+					}
+				}
+				runnableFuture.get().run();
+			}
+		}).start();
+
+		CheckpointCommitter cc = mock(CheckpointCommitter.class);
+		final CassandraTupleWriteAheadSink<Tuple0> sink = new CassandraTupleWriteAheadSink<>(
+			"abc",
+			TupleTypeInfo.of(Tuple0.class).createSerializer(new ExecutionConfig()),
+			clusterBuilder,
+			cc
+		);
+
+		OneInputStreamOperatorTestHarness<Tuple0, Tuple0> harness = new OneInputStreamOperatorTestHarness(sink);
+		harness.getEnvironment().getTaskConfiguration().setBoolean("checkpointing", true);
+
+		harness.setup();
+		sink.open();
+
+		// we should leave the loop and return false since we've seen an exception
+		assertFalse(sink.sendValues(Collections.singleton(new Tuple0()), 0L));
+
+		sink.close();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Pojo.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Pojo.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Pojo.java
new file mode 100644
index 0000000..9b331d6
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Pojo.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.cassandra;
+
+import com.datastax.driver.mapping.annotations.Column;
+import com.datastax.driver.mapping.annotations.Table;
+
+import java.io.Serializable;
+
+@Table(keyspace = "flink", name = "test")
+public class Pojo implements Serializable {
+
+	private static final long serialVersionUID = 1038054554690916991L;
+
+	@Column(name = "id")
+	private String id;
+	@Column(name = "counter")
+	private int counter;
+	@Column(name = "batch_id")
+	private int batch_id;
+
+	public Pojo(String id, int counter, int batch_id) {
+		this.id = id;
+		this.counter = counter;
+		this.batch_id = batch_id;
+	}
+
+	public String getId() {
+		return id;
+	}
+
+	public void setId(String id) {
+		this.id = id;
+	}
+
+	public int getCounter() {
+		return counter;
+	}
+
+	public void setCounter(int counter) {
+		this.counter = counter;
+	}
+
+	public int getBatch_id() {
+		return batch_id;
+	}
+
+	public void setBatch_id(int batch_id) {
+		this.batch_id = batch_id;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java
new file mode 100644
index 0000000..e1bcea9
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.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.cassandra.example;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Cluster.Builder;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSink;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import java.util.ArrayList;
+
+/**
+ * This is an example showing the to use the Pojo Cassandra Sink in the Streaming API.
+ * 
+ * Pojo's have to be annotated with datastax annotations to work with this sink.
+ *
+ * The example assumes that a table exists in a local cassandra database, according to the following query:
+ * CREATE TABLE IF NOT EXISTS test.message(body txt PRIMARY KEY)
+ */
+public class CassandraPojoSinkExample {
+	private static final ArrayList<Message> messages = new ArrayList<>(20);
+
+	static {
+		for (long i = 0; i < 20; i++) {
+			messages.add(new Message("cassandra-" + i));
+		}
+	}
+
+	public static void main(String[] args) throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStreamSource<Message> source = env.fromCollection(messages);
+
+		CassandraSink.addSink(source)
+			.setClusterBuilder(new ClusterBuilder() {
+				@Override
+				protected Cluster buildCluster(Builder builder) {
+					return builder.addContactPoint("127.0.0.1").build();
+				}
+			})
+			.build();
+
+		env.execute("Cassandra Sink example");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java
new file mode 100644
index 0000000..c6345df
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.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.cassandra.example;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Cluster.Builder;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSink;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import java.util.ArrayList;
+
+/**
+ * This is an example showing the to use the Tuple Cassandra Sink in the Streaming API.
+ *
+ * The example assumes that a table exists in a local cassandra database, according to the following query:
+ * CREATE TABLE IF NOT EXISTS test.writetuple(element1 text PRIMARY KEY, element2 int)
+ */
+public class CassandraTupleSinkExample {
+	private static final String INSERT = "INSERT INTO test.writetuple (element1, element2) VALUES (?, ?)";
+	private static final ArrayList<Tuple2<String, Integer>> collection = new ArrayList<>(20);
+
+	static {
+		for (int i = 0; i < 20; i++) {
+			collection.add(new Tuple2<>("cassandra-" + i, i));
+		}
+	}
+
+	public static void main(String[] args) throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStreamSource<Tuple2<String, Integer>> source = env.fromCollection(collection);
+
+		CassandraSink.addSink(source)
+			.setQuery(INSERT)
+			.setClusterBuilder(new ClusterBuilder() {
+				@Override
+				protected Cluster buildCluster(Builder builder) {
+					return builder.addContactPoint("127.0.0.1").build();
+				}
+			})
+			.build();
+
+		env.execute("WriteTupleIntoCassandra");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
new file mode 100644
index 0000000..811c410
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cassandra.example;
+
+import com.datastax.driver.core.Cluster;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSink;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+
+import java.util.UUID;
+
+/**
+ * This is an example showing the to use the Cassandra Sink (with write-ahead log) in the Streaming API.
+ *
+ * The example assumes that a table exists in a local cassandra database, according to the following query:
+ * CREATE TABLE example.values (id text, count int, PRIMARY KEY(id));
+ * 
+ * Important things to note are that checkpointing is enabled, a StateBackend is set and the enableWriteAheadLog() call
+ * when creating the CassandraSink.
+ */
+public class CassandraTupleWriteAheadSinkExample {
+	public static void main(String[] args) throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+		env.enableCheckpointing(1000);
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));
+		env.setStateBackend(new FsStateBackend("file:///" + System.getProperty("java.io.tmpdir") + "/flink/backend"));
+
+		CassandraSink<Tuple2<String, Integer>> sink = CassandraSink.addSink(env.addSource(new MySource()))
+			.setQuery("INSERT INTO example.values (id, counter) values (?, ?);")
+			.enableWriteAheadLog()
+			.setClusterBuilder(new ClusterBuilder() {
+				@Override
+				public Cluster buildCluster(Cluster.Builder builder) {
+					return builder.addContactPoint("127.0.0.1").build();
+				}
+			})
+			.build();
+
+		sink.name("Cassandra Sink").disableChaining().setParallelism(1).uid("hello");
+
+		env.execute();
+	}
+
+	public static class MySource implements SourceFunction<Tuple2<String, Integer>>, Checkpointed<Integer> {
+		private int counter = 0;
+		private boolean stop = false;
+
+		@Override
+		public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+			while (!stop) {
+				Thread.sleep(50);
+				ctx.collect(new Tuple2<>("" + UUID.randomUUID(), 1));
+				counter++;
+				if (counter == 100) {
+					stop = true;
+				}
+			}
+		}
+
+		@Override
+		public void cancel() {
+			stop = true;
+		}
+
+		@Override
+		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return counter;
+		}
+
+		@Override
+		public void restoreState(Integer state) throws Exception {
+			this.counter = state;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/Message.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/Message.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/Message.java
new file mode 100644
index 0000000..7524d95
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/Message.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cassandra.example;
+
+import com.datastax.driver.mapping.annotations.Column;
+import com.datastax.driver.mapping.annotations.Table;
+
+import java.io.Serializable;
+
+@Table(keyspace = "test", name = "message")
+public class Message implements Serializable {
+
+	private static final long serialVersionUID = 1123119384361005680L;
+
+	@Column(name = "body")
+	private String message;
+
+	public Message(String word) {
+		this.message = word;
+	}
+
+	public String getMessage() {
+		return message;
+	}
+
+	public void setMessage(String word) {
+		this.message = word;
+	}
+
+	public boolean equals(Object other) {
+		if (other instanceof Message) {
+			Message that = (Message) other;
+			return this.message.equals(that.message);
+		}
+		return false;
+	}
+
+	@Override
+	public int hashCode() {
+		return message.hashCode();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/de4fe3b7/flink-connectors/flink-connector-cassandra/src/test/resources/cassandra.yaml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/resources/cassandra.yaml b/flink-connectors/flink-connector-cassandra/src/test/resources/cassandra.yaml
new file mode 100644
index 0000000..0594ea3
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/resources/cassandra.yaml
@@ -0,0 +1,43 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+cluster_name: 'Test Cluster'
+commitlog_sync: 'periodic'
+commitlog_sync_period_in_ms: 10000
+commitlog_segment_size_in_mb: 16
+partitioner: 'org.apache.cassandra.dht.RandomPartitioner'
+endpoint_snitch: 'org.apache.cassandra.locator.SimpleSnitch'
+commitlog_directory: $PATH/commit'
+data_file_directories:
+    - $PATH/data'
+saved_caches_directory: $PATH/cache'
+listen_address: '127.0.0.1'
+seed_provider:
+    - class_name: 'org.apache.cassandra.locator.SimpleSeedProvider'
+      parameters:
+          - seeds: '127.0.0.1'
+native_transport_port: 9042
+
+concurrent_reads: 8
+concurrent_writes: 8
+
+auto_bootstrap: false
+auto_snapshot: false
+
+start_rpc: false
+start_native_transport: true
+native_transport_max_threads: 8

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

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

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

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

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

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

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

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