You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2015/08/27 13:25:18 UTC

[01/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Repository: flink
Updated Branches:
  refs/heads/master d08733df2 -> 541a06cfc


http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/StringSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/StringSerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/StringSerializer.java
deleted file mode 100644
index bb63638..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/StringSerializer.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.kafka.copied.common.serialization;
-
-import org.apache.kafka.copied.common.errors.SerializationException;
-
-import java.io.UnsupportedEncodingException;
-import java.util.Map;
-
-/**
- *  String encoding defaults to UTF8 and can be customized by setting the property key.serializer.encoding,
- *  value.serializer.encoding or serializer.encoding. The first two take precedence over the last.
- */
-public class StringSerializer implements Serializer<String> {
-    private String encoding = "UTF8";
-
-    @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        String propertyName = isKey ? "key.serializer.encoding" : "value.serializer.encoding";
-        Object encodingValue = configs.get(propertyName);
-        if (encodingValue == null)
-            encodingValue = configs.get("serializer.encoding");
-        if (encodingValue != null && encodingValue instanceof String)
-            encoding = (String) encodingValue;
-    }
-
-    @Override
-    public byte[] serialize(String topic, String data) {
-        try {
-            if (data == null)
-                return null;
-            else
-                return data.getBytes(encoding);
-        } catch (UnsupportedEncodingException e) {
-            throw new SerializationException("Error when serializing string to byte[] due to unsupported encoding " + encoding);
-        }
-    }
-
-    @Override
-    public void close() {
-        // nothing to do
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/AbstractIterator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/AbstractIterator.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/AbstractIterator.java
deleted file mode 100644
index b0d8b2d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/AbstractIterator.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.utils;
-
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-/**
- * A base class that simplifies implementing an iterator
- * @param <T> The type of thing we are iterating over
- */
-public abstract class AbstractIterator<T> implements Iterator<T> {
-
-    private static enum State {
-        READY, NOT_READY, DONE, FAILED
-    };
-
-    private State state = State.NOT_READY;
-    private T next;
-
-    @Override
-    public boolean hasNext() {
-        switch (state) {
-            case FAILED:
-                throw new IllegalStateException("Iterator is in failed state");
-            case DONE:
-                return false;
-            case READY:
-                return true;
-            default:
-                return maybeComputeNext();
-        }
-    }
-
-    @Override
-    public T next() {
-        if (!hasNext())
-            throw new NoSuchElementException();
-        state = State.NOT_READY;
-        if (next == null)
-            throw new IllegalStateException("Expected item but none found.");
-        return next;
-    }
-
-    @Override
-    public void remove() {
-        throw new UnsupportedOperationException("Removal not supported");
-    }
-
-    public T peek() {
-        if (!hasNext())
-            throw new NoSuchElementException();
-        return next;
-    }
-
-    protected T allDone() {
-        state = State.DONE;
-        return null;
-    }
-
-    protected abstract T makeNext();
-
-    private Boolean maybeComputeNext() {
-        state = State.FAILED;
-        next = makeNext();
-        if (state == State.DONE) {
-            return false;
-        } else {
-            state = State.READY;
-            return true;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/CollectionUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/CollectionUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/CollectionUtils.java
deleted file mode 100644
index 95dc756..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/CollectionUtils.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.kafka.copied.common.utils;
-
-import org.apache.kafka.copied.common.TopicPartition;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class CollectionUtils {
-    /**
-     * group data by topic
-     * @param data Data to be partitioned
-     * @param <T> Partition data type
-     * @return partitioned data
-     */
-    public static <T> Map<String, Map<Integer, T>> groupDataByTopic(Map<TopicPartition, T> data) {
-        Map<String, Map<Integer, T>> dataByTopic = new HashMap<String, Map<Integer, T>>();
-        for (Map.Entry<TopicPartition, T> entry: data.entrySet()) {
-            String topic = entry.getKey().topic();
-            int partition = entry.getKey().partition();
-            Map<Integer, T> topicData = dataByTopic.get(topic);
-            if (topicData == null) {
-                topicData = new HashMap<Integer, T>();
-                dataByTopic.put(topic, topicData);
-            }
-            topicData.put(partition, entry.getValue());
-        }
-        return dataByTopic;
-    }
-
-    /**
-     * group partitions by topic
-     * @param partitions
-     * @return partitions per topic
-     */
-    public static Map<String, List<Integer>> groupDataByTopic(List<TopicPartition> partitions) {
-        Map<String, List<Integer>> partitionsByTopic = new HashMap<String, List<Integer>>();
-        for (TopicPartition tp: partitions) {
-            String topic = tp.topic();
-            List<Integer> topicData = partitionsByTopic.get(topic);
-            if (topicData == null) {
-                topicData = new ArrayList<Integer>();
-                partitionsByTopic.put(topic, topicData);
-            }
-            topicData.add(tp.partition());
-        }
-        return  partitionsByTopic;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/CopyOnWriteMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/CopyOnWriteMap.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/CopyOnWriteMap.java
deleted file mode 100644
index ec85158..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/CopyOnWriteMap.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.kafka.copied.common.utils;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * A simple read-optimized map implementation that synchronizes only writes and does a full copy on each modification
- */
-public class CopyOnWriteMap<K, V> implements ConcurrentMap<K, V> {
-
-    private volatile Map<K, V> map;
-
-    public CopyOnWriteMap() {
-        this.map = Collections.emptyMap();
-    }
-
-    public CopyOnWriteMap(Map<K, V> map) {
-        this.map = Collections.unmodifiableMap(map);
-    }
-
-    @Override
-    public boolean containsKey(Object k) {
-        return map.containsKey(k);
-    }
-
-    @Override
-    public boolean containsValue(Object v) {
-        return map.containsValue(v);
-    }
-
-    @Override
-    public Set<Entry<K, V>> entrySet() {
-        return map.entrySet();
-    }
-
-    @Override
-    public V get(Object k) {
-        return map.get(k);
-    }
-
-    @Override
-    public boolean isEmpty() {
-        return map.isEmpty();
-    }
-
-    @Override
-    public Set<K> keySet() {
-        return map.keySet();
-    }
-
-    @Override
-    public int size() {
-        return map.size();
-    }
-
-    @Override
-    public Collection<V> values() {
-        return map.values();
-    }
-
-    @Override
-    public synchronized void clear() {
-        this.map = Collections.emptyMap();
-    }
-
-    @Override
-    public synchronized V put(K k, V v) {
-        Map<K, V> copy = new HashMap<K, V>(this.map);
-        V prev = copy.put(k, v);
-        this.map = Collections.unmodifiableMap(copy);
-        return prev;
-    }
-
-    @Override
-    public synchronized void putAll(Map<? extends K, ? extends V> entries) {
-        Map<K, V> copy = new HashMap<K, V>(this.map);
-        copy.putAll(entries);
-        this.map = Collections.unmodifiableMap(copy);
-    }
-
-    @Override
-    public synchronized V remove(Object key) {
-        Map<K, V> copy = new HashMap<K, V>(this.map);
-        V prev = copy.remove(key);
-        this.map = Collections.unmodifiableMap(copy);
-        return prev;
-    }
-
-    @Override
-    public synchronized V putIfAbsent(K k, V v) {
-        if (!containsKey(k))
-            return put(k, v);
-        else
-            return get(k);
-    }
-
-    @Override
-    public synchronized boolean remove(Object k, Object v) {
-        if (containsKey(k) && get(k).equals(v)) {
-            remove(k);
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public synchronized boolean replace(K k, V original, V replacement) {
-        if (containsKey(k) && get(k).equals(original)) {
-            put(k, replacement);
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public synchronized V replace(K k, V v) {
-        if (containsKey(k)) {
-            return put(k, v);
-        } else {
-            return null;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Crc32.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Crc32.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Crc32.java
deleted file mode 100644
index 944d585..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Crc32.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.kafka.copied.common.utils;
-
-import java.util.zip.Checksum;
-
-/**
- * This class was taken from Hadoop org.apache.hadoop.util.PureJavaCrc32
- * 
- * A pure-java implementation of the CRC32 checksum that uses the same polynomial as the built-in native CRC32.
- * 
- * This is to avoid the JNI overhead for certain uses of Checksumming where many small pieces of data are checksummed in
- * succession.
- * 
- * The current version is ~10x to 1.8x as fast as Sun's native java.util.zip.CRC32 in Java 1.6
- * 
- * @see java.util.zip.CRC32
- */
-public class Crc32 implements Checksum {
-
-    /**
-     * Compute the CRC32 of the byte array
-     * 
-     * @param bytes The array to compute the checksum for
-     * @return The CRC32
-     */
-    public static long crc32(byte[] bytes) {
-        return crc32(bytes, 0, bytes.length);
-    }
-
-    /**
-     * Compute the CRC32 of the segment of the byte array given by the specified size and offset
-     * 
-     * @param bytes The bytes to checksum
-     * @param offset the offset at which to begin checksumming
-     * @param size the number of bytes to checksum
-     * @return The CRC32
-     */
-    public static long crc32(byte[] bytes, int offset, int size) {
-        Crc32 crc = new Crc32();
-        crc.update(bytes, offset, size);
-        return crc.getValue();
-    }
-
-    /** the current CRC value, bit-flipped */
-    private int crc;
-
-    /** Create a new PureJavaCrc32 object. */
-    public Crc32() {
-        reset();
-    }
-
-    @Override
-    public long getValue() {
-        return (~crc) & 0xffffffffL;
-    }
-
-    @Override
-    public void reset() {
-        crc = 0xffffffff;
-    }
-
-    @Override
-    public void update(byte[] b, int off, int len) {
-        int localCrc = crc;
-
-        while (len > 7) {
-            final int c0 = (b[off + 0] ^ localCrc) & 0xff;
-            final int c1 = (b[off + 1] ^ (localCrc >>>= 8)) & 0xff;
-            final int c2 = (b[off + 2] ^ (localCrc >>>= 8)) & 0xff;
-            final int c3 = (b[off + 3] ^ (localCrc >>>= 8)) & 0xff;
-            localCrc = (T[T8_7_START + c0] ^ T[T8_6_START + c1]) ^ (T[T8_5_START + c2] ^ T[T8_4_START + c3]);
-
-            final int c4 = b[off + 4] & 0xff;
-            final int c5 = b[off + 5] & 0xff;
-            final int c6 = b[off + 6] & 0xff;
-            final int c7 = b[off + 7] & 0xff;
-
-            localCrc ^= (T[T8_3_START + c4] ^ T[T8_2_START + c5]) ^ (T[T8_1_START + c6] ^ T[T8_0_START + c7]);
-
-            off += 8;
-            len -= 8;
-        }
-
-        /* loop unroll - duff's device style */
-        switch (len) {
-            case 7:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 6:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 5:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 4:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 3:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 2:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 1:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            default:
-                /* nothing */
-        }
-
-        // Publish crc out to object
-        crc = localCrc;
-    }
-
-    @Override
-    final public void update(int b) {
-        crc = (crc >>> 8) ^ T[T8_0_START + ((crc ^ b) & 0xff)];
-    }
-
-    /**
-     * Update the CRC32 given an integer
-     */
-    final public void updateInt(int input) {
-        update((byte) (input >> 24));
-        update((byte) (input >> 16));
-        update((byte) (input >> 8));
-        update((byte) input /* >> 0 */);
-    }
-
-    /*
-     * CRC-32 lookup tables generated by the polynomial 0xEDB88320. See also TestPureJavaCrc32.Table.
-     */
-    private static final int T8_0_START = 0 * 256;
-    private static final int T8_1_START = 1 * 256;
-    private static final int T8_2_START = 2 * 256;
-    private static final int T8_3_START = 3 * 256;
-    private static final int T8_4_START = 4 * 256;
-    private static final int T8_5_START = 5 * 256;
-    private static final int T8_6_START = 6 * 256;
-    private static final int T8_7_START = 7 * 256;
-
-    private static final int[] T = new int[] {
-        /* T8_0 */
-        0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, 0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, 0x0EDB8832,
-        0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, 0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, 0x1DB71064, 0x6AB020F2,
-        0xF3B97148, 0x84BE41DE, 0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, 0x136C9856, 0x646BA8C0, 0xFD62F97A,
-        0x8A65C9EC, 0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, 0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172,
-        0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, 0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, 0x32D86CE3,
-        0x45DF5C75, 0xDCD60DCF, 0xABD13D59, 0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, 0x21B4F4B5, 0x56B3C423,
-        0xCFBA9599, 0xB8BDA50F, 0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, 0x2F6F7C87, 0x58684C11, 0xC1611DAB,
-        0xB6662D3D, 0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, 0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433,
-        0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, 0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, 0x6B6B51F4,
-        0x1C6C6162, 0x856530D8, 0xF262004E, 0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, 0x65B0D9C6, 0x12B7E950,
-        0x8BBEB8EA, 0xFCB9887C, 0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, 0x4DB26158, 0x3AB551CE, 0xA3BC0074,
-        0xD4BB30E2, 0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, 0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0,
-        0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, 0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, 0x5768B525,
-        0x206F85B3, 0xB966D409, 0xCE61E49F, 0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, 0x59B33D17, 0x2EB40D81,
-        0xB7BD5C3B, 0xC0BA6CAD, 0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, 0xEAD54739, 0x9DD277AF, 0x04DB2615,
-        0x73DC1683, 0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, 0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1,
-        0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, 0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, 0xFED41B76,
-        0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, 0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, 0xD6D6A3E8, 0xA1D1937E,
-        0x38D8C2C4, 0x4FDFF252, 0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, 0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6,
-        0x41047A60, 0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, 0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236,
-        0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, 0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, 0xC2D7FFA7,
-        0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, 0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, 0x9C0906A9, 0xEB0E363F,
-        0x72076785, 0x05005713, 0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, 0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7,
-        0x0BDBDF21, 0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, 0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777,
-        0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, 0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, 0xA00AE278,
-        0xD70DD2EE, 0x4E048354, 0x3903B3C2, 0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, 0xAED16A4A, 0xD9D65ADC,
-        0x40DF0B66, 0x37D83BF0, 0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, 0xBDBDF21C, 0xCABAC28A, 0x53B39330,
-        0x24B4A3A6, 0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, 0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94,
-        0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D,
-        /* T8_1 */
-        0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, 0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, 0xC8D98A08,
-        0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, 0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, 0x4AC21251, 0x53D92310,
-        0x78F470D3, 0x61EF4192, 0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, 0x821B9859, 0x9B00A918, 0xB02DFADB,
-        0xA936CB9A, 0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, 0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761,
-        0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, 0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, 0x39316BAE,
-        0x202A5AEF, 0x0B07092C, 0x121C386D, 0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, 0xBB2AF3F7, 0xA231C2B6,
-        0x891C9175, 0x9007A034, 0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, 0x73F379FF, 0x6AE848BE, 0x41C51B7D,
-        0x58DE2A3C, 0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, 0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2,
-        0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, 0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, 0xBABB5D54,
-        0xA3A06C15, 0x888D3FD6, 0x91960E97, 0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, 0x7262D75C, 0x6B79E61D,
-        0x4054B5DE, 0x594F849F, 0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, 0x65FD6BA7, 0x7CE65AE6, 0x57CB0925,
-        0x4ED03864, 0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, 0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C,
-        0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, 0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, 0x4B53BCF2,
-        0x52488DB3, 0x7965DE70, 0x607EEF31, 0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, 0x838A36FA, 0x9A9107BB,
-        0xB1BC5478, 0xA8A76539, 0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, 0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD,
-        0x74C20E8C, 0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, 0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484,
-        0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, 0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, 0xB9980012,
-        0xA0833153, 0x8BAE6290, 0x92B553D1, 0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, 0xAE07BCE9, 0xB71C8DA8,
-        0x9C31DE6B, 0x852AEF2A, 0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, 0x66DE36E1, 0x7FC507A0, 0x54E85463,
-        0x4DF36522, 0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, 0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B,
-        0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, 0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, 0x4870E1B4,
-        0x516BD0F5, 0x7A468336, 0x635DB277, 0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, 0xAF96124A, 0xB68D230B,
-        0x9DA070C8, 0x84BB4189, 0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, 0x674F9842, 0x7E54A903, 0x5579FAC0,
-        0x4C62CB81, 0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, 0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8,
-        0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, 0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, 0x5E7EF3EC,
-        0x4765C2AD, 0x6C48916E, 0x7553A02F, 0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, 0x96A779E4, 0x8FBC48A5,
-        0xA4911B66, 0xBD8A2A27, 0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, 0x14BCE1BD, 0x0DA7D0FC, 0x268A833F,
-        0x3F91B27E, 0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, 0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876,
-        0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72,
-        /* T8_2 */
-        0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, 0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, 0x0E1351B8,
-        0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, 0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, 0x1C26A370, 0x1DE4C947,
-        0x1FA2771E, 0x1E601D29, 0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, 0x1235F2C8, 0x13F798FF, 0x11B126A6,
-        0x10734C91, 0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, 0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9,
-        0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, 0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, 0x3157BF84,
-        0x3095D5B3, 0x32D36BEA, 0x331101DD, 0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, 0x23624D4C, 0x22A0277B,
-        0x20E69922, 0x2124F315, 0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, 0x2D711CF4, 0x2CB376C3, 0x2EF5C89A,
-        0x2F37A2AD, 0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, 0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45,
-        0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, 0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, 0x6CBC2EB0,
-        0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, 0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, 0x62AF7F08, 0x636D153F,
-        0x612BAB66, 0x60E9C151, 0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, 0x48D7CB20, 0x4915A117, 0x4B531F4E,
-        0x4A917579, 0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, 0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1,
-        0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, 0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, 0x53F8C08C,
-        0x523AAABB, 0x507C14E2, 0x51BE7ED5, 0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, 0x5DEB9134, 0x5C29FB03,
-        0x5E6F455A, 0x5FAD2F6D, 0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, 0xE63CB35C, 0xE7FED96B, 0xE5B86732,
-        0xE47A0D05, 0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, 0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD,
-        0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, 0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, 0xF300E948,
-        0xF2C2837F, 0xF0843D26, 0xF1465711, 0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, 0xD9785D60, 0xD8BA3757,
-        0xDAFC890E, 0xDB3EE339, 0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, 0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6,
-        0xD52DB281, 0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, 0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049,
-        0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, 0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, 0xCC440774,
-        0xCD866D43, 0xCFC0D31A, 0xCE02B92D, 0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, 0x96A63E9C, 0x976454AB,
-        0x9522EAF2, 0x94E080C5, 0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, 0x98B56F24, 0x99770513, 0x9B31BB4A,
-        0x9AF3D17D, 0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, 0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5,
-        0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, 0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, 0xA9E2D0A0,
-        0xA820BA97, 0xAA6604CE, 0xABA46EF9, 0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, 0xA7F18118, 0xA633EB2F,
-        0xA4755576, 0xA5B73F41, 0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, 0xB5C473D0, 0xB40619E7, 0xB640A7BE,
-        0xB782CD89, 0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, 0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31,
-        0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED,
-        /* T8_3 */
-        0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, 0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, 0xC5B428EF,
-        0x7D084F8A, 0x6FBDE064, 0xD7018701, 0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, 0x5019579F, 0xE8A530FA,
-        0xFA109F14, 0x42ACF871, 0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, 0x95AD7F70, 0x2D111815, 0x3FA4B7FB,
-        0x8718D09E, 0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, 0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0,
-        0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, 0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, 0xEAE41086,
-        0x525877E3, 0x40EDD80D, 0xF851BF68, 0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, 0x7F496FF6, 0xC7F50893,
-        0xD540A77D, 0x6DFCC018, 0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, 0xBAFD4719, 0x0241207C, 0x10F48F92,
-        0xA848E8F7, 0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, 0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084,
-        0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, 0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, 0xCB0D0FA2,
-        0x73B168C7, 0x6104C729, 0xD9B8A04C, 0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, 0x0EB9274D, 0xB6054028,
-        0xA4B0EFC6, 0x1C0C88A3, 0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, 0x3B26F703, 0x839A9066, 0x912F3F88,
-        0x299358ED, 0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, 0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002,
-        0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, 0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, 0xE45D37CB,
-        0x5CE150AE, 0x4E54FF40, 0xF6E89825, 0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, 0x21E91F24, 0x99557841,
-        0x8BE0D7AF, 0x335CB0CA, 0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, 0x623B216C, 0xDA874609, 0xC832E9E7,
-        0x708E8E82, 0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, 0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D,
-        0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, 0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, 0x78F4C94B,
-        0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, 0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, 0x4D6B1905, 0xF5D77E60,
-        0xE762D18E, 0x5FDEB6EB, 0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, 0x88DF31EA, 0x3063568F, 0x22D6F961,
-        0x9A6A9E04, 0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, 0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174,
-        0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, 0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, 0x57A4F122,
-        0xEF189647, 0xFDAD39A9, 0x45115ECC, 0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, 0xF92F7951, 0x41931E34,
-        0x5326B1DA, 0xEB9AD6BF, 0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, 0x3C9B51BE, 0x842736DB, 0x96929935,
-        0x2E2EFE50, 0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, 0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120,
-        0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, 0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, 0xD67F4138,
-        0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, 0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, 0x13CB69D7, 0xAB770EB2,
-        0xB9C2A15C, 0x017EC639, 0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, 0x866616A7, 0x3EDA71C2, 0x2C6FDE2C,
-        0x94D3B949, 0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, 0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6,
-        0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1,
-        /* T8_4 */
-        0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, 0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, 0x30704BC1,
-        0x0D106271, 0x4AB018A1, 0x77D03111, 0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, 0x60E09782, 0x5D80BE32,
-        0x1A20C4E2, 0x2740ED52, 0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, 0x5090DC43, 0x6DF0F5F3, 0x2A508F23,
-        0x1730A693, 0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, 0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4,
-        0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, 0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, 0x0431C205,
-        0x3951EBB5, 0x7EF19165, 0x4391B8D5, 0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, 0x54A11E46, 0x69C137F6,
-        0x2E614D26, 0x13016496, 0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, 0x64D15587, 0x59B17C37, 0x1E1106E7,
-        0x23712F57, 0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, 0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459,
-        0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, 0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, 0x3813CFCB,
-        0x0573E67B, 0x42D39CAB, 0x7FB3B51B, 0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, 0x0863840A, 0x3503ADBA,
-        0x72A3D76A, 0x4FC3FEDA, 0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, 0x9932774D, 0xA4525EFD, 0xE3F2242D,
-        0xDE920D9D, 0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, 0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C,
-        0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, 0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, 0x0C52460F,
-        0x31326FBF, 0x7692156F, 0x4BF23CDF, 0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, 0x3C220DCE, 0x0142247E,
-        0x46E25EAE, 0x7B82771E, 0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, 0x44661652, 0x79063FE2, 0x3EA64532,
-        0x03C66C82, 0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, 0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743,
-        0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, 0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, 0xE1766CD1,
-        0xDC164561, 0x9BB63FB1, 0xA6D61601, 0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, 0x70279F96, 0x4D47B626,
-        0x0AE7CCF6, 0x3787E546, 0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, 0x4057D457, 0x7D37FDE7, 0x3A978737,
-        0x07F7AE87, 0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, 0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4,
-        0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, 0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, 0xD537E515,
-        0xE857CCA5, 0xAFF7B675, 0x92979FC5, 0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, 0x1C954E1B, 0x21F567AB,
-        0x66551D7B, 0x5B3534CB, 0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, 0x2CE505DA, 0x11852C6A, 0x562556BA,
-        0x6B457F0A, 0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, 0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349,
-        0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, 0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, 0x28D4C7DF,
-        0x15B4EE6F, 0x521494BF, 0x6F74BD0F, 0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, 0x18A48C1E, 0x25C4A5AE,
-        0x6264DF7E, 0x5F04F6CE, 0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, 0x4834505D, 0x755479ED, 0x32F4033D,
-        0x0F942A8D, 0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, 0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C,
-        0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C,
-        /* T8_5 */
-        0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, 0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, 0xEC53826D,
-        0x270F51C8, 0xA19B2366, 0x6AC7F0C3, 0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, 0x03D6029B, 0xC88AD13E,
-        0x4E1EA390, 0x85427035, 0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, 0xEF8580F6, 0x24D95353, 0xA24D21FD,
-        0x6911F258, 0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, 0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798,
-        0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, 0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, 0x706EC54D,
-        0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, 0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, 0x9FEB45BB, 0x54B7961E,
-        0xD223E4B0, 0x197F3715, 0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, 0x73B8C7D6, 0xB8E41473, 0x3E7066DD,
-        0xF52CB578, 0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, 0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4,
-        0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, 0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, 0x0C8E08F7,
-        0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, 0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, 0xE0DD8A9A, 0x2B81593F,
-        0xAD152B91, 0x6649F834, 0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, 0x08F40F5A, 0xC3A8DCFF, 0x453CAE51,
-        0x8E607DF4, 0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, 0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99,
-        0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, 0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, 0x90B34FD7,
-        0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, 0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, 0x7CE0CDBA, 0xB7BC1E1F,
-        0x31286CB1, 0xFA74BF14, 0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, 0x852156CE, 0x4E7D856B, 0xC8E9F7C5,
-        0x03B52460, 0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, 0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D,
-        0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, 0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, 0xF135942E,
-        0x3A69478B, 0xBCFD3525, 0x77A1E680, 0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, 0x191C11EE, 0xD240C24B,
-        0x54D4B0E5, 0x9F886340, 0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, 0xF54F9383, 0x3E134026, 0xB8873288,
-        0x73DBE12D, 0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, 0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB,
-        0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, 0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, 0x6D08D30E,
-        0xA65400AB, 0x20C07205, 0xEB9CA1A0, 0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, 0x8A795CA2, 0x41258F07,
-        0xC7B1FDA9, 0x0CED2E0C, 0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, 0x662ADECF, 0xAD760D6A, 0x2BE27FC4,
-        0xE0BEAC61, 0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, 0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97,
-        0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, 0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, 0x16441B82,
-        0xDD18C827, 0x5B8CBA89, 0x90D0692C, 0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, 0xFA1799EF, 0x314B4A4A,
-        0xB7DF38E4, 0x7C83EB41, 0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, 0x15921919, 0xDECECABC, 0x585AB812,
-        0x93066BB7, 0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, 0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA,
-        0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC,
-        /* T8_6 */
-        0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, 0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, 0x33EF4E67,
-        0x959845D3, 0xA4705F4E, 0x020754FA, 0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, 0x67DE9CCE, 0xC1A9977A,
-        0xF0418DE7, 0x56368653, 0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, 0x5431D2A9, 0xF246D91D, 0xC3AEC380,
-        0x65D9C834, 0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, 0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301,
-        0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, 0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, 0x081D53E8,
-        0xAE6A585C, 0x9F8242C1, 0x39F54975, 0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, 0x5C2C8141, 0xFA5B8AF5,
-        0xCBB39068, 0x6DC49BDC, 0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, 0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F,
-        0x5E2BD5BB, 0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, 0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7,
-        0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, 0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, 0x23D5E9B7,
-        0x85A2E203, 0xB44AF89E, 0x123DF32A, 0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, 0x103AA7D0, 0xB64DAC64,
-        0x87A5B6F9, 0x21D2BD4D, 0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, 0x8BB64CE5, 0x2DC14751, 0x1C295DCC,
-        0xBA5E5678, 0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, 0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F,
-        0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, 0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, 0x1827F438,
-        0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, 0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, 0x2BC8BA5F, 0x8DBFB1EB,
-        0xBC57AB76, 0x1A20A0C2, 0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, 0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8,
-        0x4DB1D47C, 0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, 0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B,
-        0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, 0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, 0xDC27385B,
-        0x7A5033EF, 0x4BB82972, 0xEDCF22C6, 0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, 0x47ABD36E, 0xE1DCD8DA,
-        0xD034C247, 0x7643C9F3, 0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, 0x74449D09, 0xD23396BD, 0xE3DB8C20,
-        0x45AC8794, 0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, 0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D,
-        0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, 0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, 0xE7D525D4,
-        0x41A22E60, 0x704A34FD, 0xD63D3F49, 0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, 0x3852BB98, 0x9E25B02C,
-        0xAFCDAAB1, 0x09BAA105, 0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, 0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6,
-        0x3A55EF62, 0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, 0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB,
-        0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, 0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, 0x03A0A617,
-        0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, 0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, 0x304FE870, 0x9638E3C4,
-        0xA7D0F959, 0x01A7F2ED, 0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, 0x647E3AD9, 0xC209316D, 0xF3E12BF0,
-        0x55962044, 0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, 0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23,
-        0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30,
-        /* T8_7 */
-        0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, 0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, 0xD3E51BB5,
-        0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, 0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, 0x7CBB312B, 0xB01131B5,
-        0x3E9E3656, 0xF23436C8, 0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, 0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3,
-        0x21D12D7D, 0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, 0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5,
-        0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, 0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, 0xAED97719,
-        0x62737787, 0xECFC7064, 0x205670FA, 0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, 0x01875D87, 0xCD2D5D19,
-        0x43A25AFA, 0x8F085A64, 0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, 0xD2624632, 0x1EC846AC, 0x9047414F,
-        0x5CED41D1, 0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, 0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4,
-        0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, 0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, 0x5526F3C6,
-        0x998CF358, 0x1703F4BB, 0xDBA9F425, 0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, 0x86C3E873, 0x4A69E8ED,
-        0xC4E6EF0E, 0x084CEF90, 0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, 0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6,
-        0x5E64A758, 0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, 0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED,
-        0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, 0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, 0x281A9F6A,
-        0xE4B09FF4, 0x6A3F9817, 0xA6959889, 0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, 0xFBFF84DF, 0x37558441,
-        0xB9DA83A2, 0x7570833C, 0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, 0xD7718B20, 0x1BDB8BBE, 0x95548C5D,
-        0x59FE8CC3, 0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, 0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776,
-        0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, 0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, 0xFC65AF44,
-        0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, 0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, 0xAA4DE78C, 0x66E7E712,
-        0xE868E0F1, 0x24C2E06F, 0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, 0x79A8FC39, 0xB502FCA7, 0x3B8DFB44,
-        0xF727FBDA, 0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, 0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144,
-        0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, 0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, 0x8159C3E8,
-        0x4DF3C376, 0xC37CC495, 0x0FD6C40B, 0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, 0xFEEC49CD, 0x32464953,
-        0xBCC94EB0, 0x70634E2E, 0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, 0x2D095278, 0xE1A352E6, 0x6F2C5505,
-        0xA386559B, 0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, 0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05,
-        0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, 0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, 0x83D02561,
-        0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, 0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, 0x50353ED4, 0x9C9F3E4A,
-        0x121039A9, 0xDEBA3937, 0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, 0xFF6B144A, 0x33C114D4, 0xBD4E1337,
-        0x71E413A9, 0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, 0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C,
-        0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6 };
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/KafkaThread.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/KafkaThread.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/KafkaThread.java
deleted file mode 100644
index 3367b81..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/KafkaThread.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.kafka.copied.common.utils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A wrapper for Thread that sets things up nicely
- */
-public class KafkaThread extends Thread {
-
-    private final Logger log = LoggerFactory.getLogger(getClass());
-
-    public KafkaThread(final String name, Runnable runnable, boolean daemon) {
-        super(runnable, name);
-        setDaemon(daemon);
-        setUncaughtExceptionHandler(new UncaughtExceptionHandler() {
-            public void uncaughtException(Thread t, Throwable e) {
-                log.error("Uncaught exception in " + name + ": ", e);
-            }
-        });
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/SystemTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/SystemTime.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/SystemTime.java
deleted file mode 100644
index 598cd80..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/SystemTime.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.kafka.copied.common.utils;
-
-/**
- * A time implementation that uses the system clock and sleep call
- */
-public class SystemTime implements Time {
-
-    @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) {
-            // just wake up early
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Time.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Time.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Time.java
deleted file mode 100644
index fffd869..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Time.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.kafka.copied.common.utils;
-
-/**
- * An interface abstracting the clock to use in unit testing classes that make use of clock time
- */
-public interface Time {
-
-    /**
-     * The current time in milliseconds
-     */
-    public long milliseconds();
-
-    /**
-     * The current time in nanoseconds
-     */
-    public long nanoseconds();
-
-    /**
-     * Sleep for the given number of milliseconds
-     */
-    public void sleep(long ms);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Utils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Utils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Utils.java
deleted file mode 100755
index ad2faa5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/utils/Utils.java
+++ /dev/null
@@ -1,497 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.utils;
-
-import org.apache.kafka.copied.common.KafkaException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.io.UnsupportedEncodingException;
-import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Properties;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public class Utils {
-
-    // This matches URIs of formats: host:port and protocol:\\host:port
-    // IPv6 is supported with [ip] pattern
-    private static final Pattern HOST_PORT_PATTERN = Pattern.compile(".*?\\[?([0-9a-z\\-.:]*)\\]?:([0-9]+)");
-
-    public static final String NL = System.getProperty("line.separator");
-
-    private static final Logger log = LoggerFactory.getLogger(Utils.class);
-
-    /**
-     * Turn the given UTF8 byte array into a string
-     * 
-     * @param bytes The byte array
-     * @return The string
-     */
-    public static String utf8(byte[] bytes) {
-        try {
-            return new String(bytes, "UTF8");
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException("This shouldn't happen.", e);
-        }
-    }
-
-    /**
-     * Turn a string into a utf8 byte[]
-     * 
-     * @param string The string
-     * @return The byte[]
-     */
-    public static byte[] utf8(String string) {
-        try {
-            return string.getBytes("UTF8");
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException("This shouldn't happen.", e);
-        }
-    }
-
-    /**
-     * Read an unsigned integer from the current position in the buffer, incrementing the position by 4 bytes
-     * 
-     * @param buffer The buffer to read from
-     * @return The integer read, as a long to avoid signedness
-     */
-    public static long readUnsignedInt(ByteBuffer buffer) {
-        return buffer.getInt() & 0xffffffffL;
-    }
-
-    /**
-     * Read an unsigned integer from the given position without modifying the buffers position
-     * 
-     * @param buffer the buffer to read from
-     * @param index the index from which to read the integer
-     * @return The integer read, as a long to avoid signedness
-     */
-    public static long readUnsignedInt(ByteBuffer buffer, int index) {
-        return buffer.getInt(index) & 0xffffffffL;
-    }
-
-    /**
-     * Read an unsigned integer stored in little-endian format from the {@link InputStream}.
-     * 
-     * @param in The stream to read from
-     * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS)
-     */
-    public static int readUnsignedIntLE(InputStream in) throws IOException {
-        return (in.read() << 8 * 0) 
-             | (in.read() << 8 * 1)
-             | (in.read() << 8 * 2)
-             | (in.read() << 8 * 3);
-    }
-
-    /**
-     * Read an unsigned integer stored in little-endian format from a byte array
-     * at a given offset.
-     * 
-     * @param buffer The byte array to read from
-     * @param offset The position in buffer to read from
-     * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS)
-     */
-    public static int readUnsignedIntLE(byte[] buffer, int offset) {
-        return (buffer[offset++] << 8 * 0)
-             | (buffer[offset++] << 8 * 1)
-             | (buffer[offset++] << 8 * 2)
-             | (buffer[offset]   << 8 * 3);
-    }
-
-    /**
-     * Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
-     * 
-     * @param buffer The buffer to write to
-     * @param value The value to write
-     */
-    public static void writetUnsignedInt(ByteBuffer buffer, long value) {
-        buffer.putInt((int) (value & 0xffffffffL));
-    }
-
-    /**
-     * Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
-     * 
-     * @param buffer The buffer to write to
-     * @param index The position in the buffer at which to begin writing
-     * @param value The value to write
-     */
-    public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) {
-        buffer.putInt(index, (int) (value & 0xffffffffL));
-    }
-
-    /**
-     * Write an unsigned integer in little-endian format to the {@link OutputStream}.
-     * 
-     * @param out The stream to write to
-     * @param value The value to write
-     */
-    public static void writeUnsignedIntLE(OutputStream out, int value) throws IOException {
-        out.write(value >>> 8 * 0);
-        out.write(value >>> 8 * 1);
-        out.write(value >>> 8 * 2);
-        out.write(value >>> 8 * 3);
-    }
-
-    /**
-     * Write an unsigned integer in little-endian format to a byte array
-     * at a given offset.
-     * 
-     * @param buffer The byte array to write to
-     * @param offset The position in buffer to write to
-     * @param value The value to write
-     */
-    public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) {
-        buffer[offset++] = (byte) (value >>> 8 * 0);
-        buffer[offset++] = (byte) (value >>> 8 * 1);
-        buffer[offset++] = (byte) (value >>> 8 * 2);
-        buffer[offset]   = (byte) (value >>> 8 * 3);
-    }
-
-
-    /**
-     * Get the absolute value of the given number. If the number is Int.MinValue return 0. This is different from
-     * java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!).
-     */
-    public static int abs(int n) {
-        return (n == Integer.MIN_VALUE) ? 0 : Math.abs(n);
-    }
-
-    /**
-     * Get the minimum of some long values.
-     * @param first Used to ensure at least one value
-     * @param rest The rest of longs to compare
-     * @return The minimum of all passed argument.
-     */
-    public static long min(long first, long ... rest) {
-        long min = first;
-        for (int i = 0; i < rest.length; i++) {
-            if (rest[i] < min)
-                min = rest[i];
-        }
-        return min;
-    }
-
-    /**
-     * Get the length for UTF8-encoding a string without encoding it first
-     * 
-     * @param s The string to calculate the length for
-     * @return The length when serialized
-     */
-    public static int utf8Length(CharSequence s) {
-        int count = 0;
-        for (int i = 0, len = s.length(); i < len; i++) {
-            char ch = s.charAt(i);
-            if (ch <= 0x7F) {
-                count++;
-            } else if (ch <= 0x7FF) {
-                count += 2;
-            } else if (Character.isHighSurrogate(ch)) {
-                count += 4;
-                ++i;
-            } else {
-                count += 3;
-            }
-        }
-        return count;
-    }
-
-    /**
-     * Read the given byte buffer into a byte array
-     */
-    public static byte[] toArray(ByteBuffer buffer) {
-        return toArray(buffer, 0, buffer.limit());
-    }
-
-    /**
-     * Read a byte array from the given offset and size in the buffer
-     */
-    public static byte[] toArray(ByteBuffer buffer, int offset, int size) {
-        byte[] dest = new byte[size];
-        if (buffer.hasArray()) {
-            System.arraycopy(buffer.array(), buffer.arrayOffset() + offset, dest, 0, size);
-        } else {
-            int pos = buffer.position();
-            buffer.get(dest);
-            buffer.position(pos);
-        }
-        return dest;
-    }
-
-    /**
-     * Check that the parameter t is not null
-     * 
-     * @param t The object to check
-     * @return t if it isn't null
-     * @throws NullPointerException if t is null.
-     */
-    public static <T> T notNull(T t) {
-        if (t == null)
-            throw new NullPointerException();
-        else
-            return t;
-    }
-
-    /**
-     * Sleep for a bit
-     * @param ms The duration of the sleep
-     */
-    public static void sleep(long ms) {
-        try {
-            Thread.sleep(ms);
-        } catch (InterruptedException e) {
-            // this is okay, we just wake up early
-        }
-    }
-
-    /**
-     * Instantiate the class
-     */
-    public static Object newInstance(Class<?> c) {
-        try {
-            return c.newInstance();
-        } catch (IllegalAccessException e) {
-            throw new KafkaException("Could not instantiate class " + c.getName(), e);
-        } catch (InstantiationException e) {
-            throw new KafkaException("Could not instantiate class " + c.getName() + " Does it have a public no-argument constructor?", e);
-        }
-    }
-
-    /**
-     * Generates 32 bit murmur2 hash from byte array
-     * @param data byte array to hash
-     * @return 32 bit hash of the given array
-     */
-    public static int murmur2(final byte[] data) {
-        int length = data.length;
-        int seed = 0x9747b28c;
-        // 'm' and 'r' are mixing constants generated offline.
-        // They're not really 'magic', they just happen to work well.
-        final int m = 0x5bd1e995;
-        final int r = 24;
-
-        // Initialize the hash to a random value
-        int h = seed ^ length;
-        int length4 = length / 4;
-
-        for (int i = 0; i < length4; i++) {
-            final int i4 = i * 4;
-            int k = (data[i4 + 0] & 0xff) + ((data[i4 + 1] & 0xff) << 8) + ((data[i4 + 2] & 0xff) << 16) + ((data[i4 + 3] & 0xff) << 24);
-            k *= m;
-            k ^= k >>> r;
-            k *= m;
-            h *= m;
-            h ^= k;
-        }
-
-        // Handle the last few bytes of the input array
-        switch (length % 4) {
-            case 3:
-                h ^= (data[(length & ~3) + 2] & 0xff) << 16;
-            case 2:
-                h ^= (data[(length & ~3) + 1] & 0xff) << 8;
-            case 1:
-                h ^= data[length & ~3] & 0xff;
-                h *= m;
-        }
-
-        h ^= h >>> 13;
-        h *= m;
-        h ^= h >>> 15;
-
-        return h;
-    }
-
-    /**
-     * Extracts the hostname from a "host:port" address string.
-     * @param address address string to parse
-     * @return hostname or null if the given address is incorrect
-     */
-    public static String getHost(String address) {
-        Matcher matcher = HOST_PORT_PATTERN.matcher(address);
-        return matcher.matches() ? matcher.group(1) : null;
-    }
-
-    /**
-     * Extracts the port number from a "host:port" address string.
-     * @param address address string to parse
-     * @return port number or null if the given address is incorrect
-     */
-    public static Integer getPort(String address) {
-        Matcher matcher = HOST_PORT_PATTERN.matcher(address);
-        return matcher.matches() ? Integer.parseInt(matcher.group(2)) : null;
-    }
-
-    /**
-     * Formats hostname and port number as a "host:port" address string,
-     * surrounding IPv6 addresses with braces '[', ']'
-     * @param host hostname
-     * @param port port number
-     * @return address string
-     */
-    public static String formatAddress(String host, Integer port) {
-        return host.contains(":")
-                ? "[" + host + "]:" + port // IPv6
-                : host + ":" + port;
-    }
-
-    /**
-     * Create a string representation of an array joined by the given separator
-     * @param strs The array of items
-     * @param seperator The separator
-     * @return The string representation.
-     */
-    public static <T> String join(T[] strs, String seperator) {
-        return join(Arrays.asList(strs), seperator);
-    }
-    
-    /**
-     * Create a string representation of a list joined by the given separator
-     * @param list The list of items
-     * @param seperator The separator
-     * @return The string representation.
-     */
-    public static <T> String join(Collection<T> list, String seperator) {
-        StringBuilder sb = new StringBuilder();
-        Iterator<T> iter = list.iterator();
-        while (iter.hasNext()) {
-            sb.append(iter.next());
-            if (iter.hasNext())
-                sb.append(seperator);  
-        }
-        return sb.toString();
-    }
-
-    /**
-     * Read a properties file from the given path
-     * @param filename The path of the file to read
-     */
-    public static Properties loadProps(String filename) throws IOException, FileNotFoundException {
-        Properties props = new Properties();
-        InputStream propStream = null;
-        try {
-            propStream = new FileInputStream(filename);
-            props.load(propStream);
-        } finally {
-            if (propStream != null)
-                propStream.close();
-        }
-        return props;
-    }
-
-    /**
-     * Get the stack trace from an exception as a string
-     */
-    public static String stackTrace(Throwable e) {
-        StringWriter sw = new StringWriter();
-        PrintWriter pw = new PrintWriter(sw);
-        e.printStackTrace(pw);
-        return sw.toString();
-    }
-
-    /**
-     * Create a new thread
-     * @param name The name of the thread
-     * @param runnable The work for the thread to do
-     * @param daemon Should the thread block JVM shutdown?
-     * @return The unstarted thread
-     */
-    public static Thread newThread(String name, Runnable runnable, Boolean daemon) {
-        Thread thread = new Thread(runnable, name);
-        thread.setDaemon(daemon);
-        thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
-            public void uncaughtException(Thread t, Throwable e) {
-                log.error("Uncaught exception in thread '" + t.getName() + "':", e);
-            }
-        });
-        return thread;
-    }
-
-    /**
-     * Create a daemon thread
-     * @param name The name of the thread
-     * @param runnable The runnable to execute in the background
-     * @return The unstarted thread
-     */
-    public static Thread daemonThread(String name, Runnable runnable) {
-        return newThread(name, runnable, true);
-    }
-
-    /**
-     * Print an error message and shutdown the JVM
-     * @param message The error message
-     */
-    public static void croak(String message) {
-        System.err.println(message);
-        System.exit(1);
-    }
-
-    /**
-     * Read a buffer into a Byte array for the given offset and length
-     */
-    public static byte[] readBytes(ByteBuffer buffer, int offset, int length) {
-        byte[] dest = new byte[length];
-        if (buffer.hasArray()) {
-            System.arraycopy(buffer.array(), buffer.arrayOffset() + offset, dest, 0, length);
-        } else {
-            buffer.mark();
-            buffer.position(offset);
-            buffer.get(dest, 0, length);
-            buffer.reset();
-        }
-        return dest;
-    }
-
-    /**
-     * Read the given byte buffer into a Byte array
-     */
-    public static byte[] readBytes(ByteBuffer buffer) {
-        return Utils.readBytes(buffer, 0, buffer.limit());
-    }
-
-    /**
-     * Attempt to read a file as a string
-     * @throws IOException 
-     */
-    public static String readFileAsString(String path, Charset charset) throws IOException {
-        if (charset == null) charset = Charset.defaultCharset();
-        FileInputStream stream = new FileInputStream(new File(path));
-        String result = new String();
-        try {
-            FileChannel fc = stream.getChannel();
-            MappedByteBuffer bb = fc.map(FileChannel.MapMode.READ_ONLY, 0, fc.size());
-            result = charset.decode(bb).toString();
-        } finally {
-            stream.close();
-        }
-        return result;
-    }
-
-    public static String readFileAsString(String path) throws IOException {
-        return Utils.readFileAsString(path, Charset.defaultCharset());
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/tools/maven/suppressions.xml
----------------------------------------------------------------------
diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml
index 3357fbf..d76ed47 100644
--- a/tools/maven/suppressions.xml
+++ b/tools/maven/suppressions.xml
@@ -25,5 +25,5 @@ under the License.
 <suppressions>
 		<suppress files="org[\\/]apache[\\/]flink[\\/]api[\\/]io[\\/]avro[\\/]example[\\/]User.java" checks="[a-zA-Z0-9]*"/>
 		<suppress files="org[\\/]apache[\\/]flink[\\/]api[\\/]io[\\/]avro[\\/]generated[\\/].*.java" checks="[a-zA-Z0-9]*"/>
-		<suppress files="org[\\/]apache[\\/]kafka[\\/]copied.*.java" checks="[a-zA-Z0-9]*"/>
-</suppressions>
\ No newline at end of file
+		<suppress files="org[\\/]apache[\\/]flink[\\/]kafka_backport[\\/].*.java" checks="[a-zA-Z0-9]*"/>
+</suppressions>


[27/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Refactor, cleanup, and fix kafka consumers

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

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


[21/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Coordinator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Coordinator.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Coordinator.java
new file mode 100644
index 0000000..1746c22
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Coordinator.java
@@ -0,0 +1,791 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer.internals;
+
+import org.apache.flink.kafka_backport.clients.ClientResponse;
+import org.apache.flink.kafka_backport.clients.consumer.CommitType;
+import org.apache.flink.kafka_backport.clients.consumer.ConsumerCommitCallback;
+import org.apache.flink.kafka_backport.common.KafkaException;
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.errors.DisconnectException;
+import org.apache.flink.kafka_backport.common.metrics.Measurable;
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+import org.apache.flink.kafka_backport.common.metrics.Metrics;
+import org.apache.flink.kafka_backport.common.metrics.Sensor;
+import org.apache.flink.kafka_backport.common.metrics.stats.Avg;
+import org.apache.flink.kafka_backport.common.metrics.stats.Count;
+import org.apache.flink.kafka_backport.common.metrics.stats.Max;
+import org.apache.flink.kafka_backport.common.metrics.stats.Rate;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+import org.apache.flink.kafka_backport.common.requests.ConsumerMetadataRequest;
+import org.apache.flink.kafka_backport.common.requests.ConsumerMetadataResponse;
+import org.apache.flink.kafka_backport.common.requests.HeartbeatRequest;
+import org.apache.flink.kafka_backport.common.requests.HeartbeatResponse;
+import org.apache.flink.kafka_backport.common.requests.JoinGroupRequest;
+import org.apache.flink.kafka_backport.common.requests.JoinGroupResponse;
+import org.apache.flink.kafka_backport.common.requests.OffsetCommitRequest;
+import org.apache.flink.kafka_backport.common.requests.OffsetCommitResponse;
+import org.apache.flink.kafka_backport.common.requests.OffsetFetchRequest;
+import org.apache.flink.kafka_backport.common.requests.OffsetFetchResponse;
+import org.apache.flink.kafka_backport.common.utils.Time;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class manages the coordination process with the consumer coordinator.
+ */
+public final class Coordinator {
+
+    private static final Logger log = LoggerFactory.getLogger(Coordinator.class);
+
+    private final ConsumerNetworkClient client;
+    private final Time time;
+    private final String groupId;
+    private final Heartbeat heartbeat;
+    private final HeartbeatTask heartbeatTask;
+    private final int sessionTimeoutMs;
+    private final String assignmentStrategy;
+    private final SubscriptionState subscriptions;
+    private final CoordinatorMetrics sensors;
+    private final long requestTimeoutMs;
+    private final long retryBackoffMs;
+    private final RebalanceCallback rebalanceCallback;
+    private Node consumerCoordinator;
+    private String consumerId;
+    private int generation;
+
+
+    /**
+     * Initialize the coordination manager.
+     */
+    public Coordinator(ConsumerNetworkClient client,
+                       String groupId,
+                       int sessionTimeoutMs,
+                       String assignmentStrategy,
+                       SubscriptionState subscriptions,
+                       Metrics metrics,
+                       String metricGrpPrefix,
+                       Map<String, String> metricTags,
+                       Time time,
+                       long requestTimeoutMs,
+                       long retryBackoffMs,
+                       RebalanceCallback rebalanceCallback) {
+
+        this.client = client;
+        this.time = time;
+        this.generation = -1;
+        this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
+        this.groupId = groupId;
+        this.consumerCoordinator = null;
+        this.subscriptions = subscriptions;
+        this.sessionTimeoutMs = sessionTimeoutMs;
+        this.assignmentStrategy = assignmentStrategy;
+        this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds());
+        this.heartbeatTask = new HeartbeatTask();
+        this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
+        this.requestTimeoutMs = requestTimeoutMs;
+        this.retryBackoffMs = retryBackoffMs;
+        this.rebalanceCallback = rebalanceCallback;
+    }
+
+    /**
+     * Refresh the committed offsets for provided partitions.
+     */
+    public void refreshCommittedOffsetsIfNeeded() {
+        if (subscriptions.refreshCommitsNeeded()) {
+            Map<TopicPartition, Long> offsets = fetchCommittedOffsets(subscriptions.assignedPartitions());
+            for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet()) {
+                TopicPartition tp = entry.getKey();
+                this.subscriptions.committed(tp, entry.getValue());
+            }
+            this.subscriptions.commitsRefreshed();
+        }
+    }
+
+    /**
+     * Fetch the current committed offsets from the coordinator for a set of partitions.
+     * @param partitions The partitions to fetch offsets for
+     * @return A map from partition to the committed offset
+     */
+    public Map<TopicPartition, Long> fetchCommittedOffsets(Set<TopicPartition> partitions) {
+        while (true) {
+            ensureCoordinatorKnown();
+            ensurePartitionAssignment();
+
+            // contact coordinator to fetch committed offsets
+            RequestFuture<Map<TopicPartition, Long>> future = sendOffsetFetchRequest(partitions);
+            client.poll(future);
+
+            if (future.succeeded())
+                return future.value();
+
+            if (!future.isRetriable())
+                throw future.exception();
+
+            Utils.sleep(retryBackoffMs);
+        }
+    }
+
+    /**
+     * Ensure that we have a valid partition assignment from the coordinator.
+     */
+    public void ensurePartitionAssignment() {
+        if (!subscriptions.partitionAssignmentNeeded())
+            return;
+
+        // execute the user's callback before rebalance
+        log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions());
+        try {
+            Set<TopicPartition> revoked = new HashSet<TopicPartition>(subscriptions.assignedPartitions());
+            rebalanceCallback.onPartitionsRevoked(revoked);
+        } catch (Exception e) {
+            log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
+                    + " failed on partition revocation: ", e);
+        }
+
+        reassignPartitions();
+
+        // execute the user's callback after rebalance
+        log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions());
+        try {
+            Set<TopicPartition> assigned = new HashSet<TopicPartition>(subscriptions.assignedPartitions());
+            rebalanceCallback.onPartitionsAssigned(assigned);
+        } catch (Exception e) {
+            log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
+                    + " failed on partition assignment: ", e);
+        }
+    }
+
+    private void reassignPartitions() {
+        while (subscriptions.partitionAssignmentNeeded()) {
+            ensureCoordinatorKnown();
+
+            // ensure that there are no pending requests to the coordinator. This is important
+            // in particular to avoid resending a pending JoinGroup request.
+            if (client.pendingRequestCount(this.consumerCoordinator) > 0) {
+                client.awaitPendingRequests(this.consumerCoordinator);
+                continue;
+            }
+
+            RequestFuture<Void> future = sendJoinGroupRequest();
+            client.poll(future);
+
+            if (future.failed()) {
+                if (!future.isRetriable())
+                    throw future.exception();
+                Utils.sleep(retryBackoffMs);
+            }
+        }
+    }
+
+    /**
+     * Block until the coordinator for this group is known.
+     */
+    public void ensureCoordinatorKnown() {
+        while (coordinatorUnknown()) {
+            RequestFuture<Void> future = sendConsumerMetadataRequest();
+            client.poll(future, requestTimeoutMs);
+
+            if (future.failed())
+                client.awaitMetadataUpdate();
+        }
+    }
+
+    /**
+     * Commit offsets. This call blocks (regardless of commitType) until the coordinator
+     * can receive the commit request. Once the request has been made, however, only the
+     * synchronous commits will wait for a successful response from the coordinator.
+     * @param offsets Offsets to commit.
+     * @param commitType Commit policy
+     * @param callback Callback to be executed when the commit request finishes
+     */
+    public void commitOffsets(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
+        if (commitType == CommitType.ASYNC)
+            commitOffsetsAsync(offsets, callback);
+        else
+            commitOffsetsSync(offsets, callback);
+    }
+
+    private class HeartbeatTask implements DelayedTask {
+
+        public void reset() {
+            // start or restart the heartbeat task to be executed at the next chance
+            long now = time.milliseconds();
+            heartbeat.resetSessionTimeout(now);
+            client.unschedule(this);
+            client.schedule(this, now);
+        }
+
+        @Override
+        public void run(final long now) {
+            if (!subscriptions.partitionsAutoAssigned() ||
+                    subscriptions.partitionAssignmentNeeded() ||
+                    coordinatorUnknown())
+                // no need to send if we're not using auto-assignment or if we are
+                // awaiting a rebalance
+                return;
+
+            if (heartbeat.sessionTimeoutExpired(now)) {
+                // we haven't received a successful heartbeat in one session interval
+                // so mark the coordinator dead
+                coordinatorDead();
+                return;
+            }
+
+            if (!heartbeat.shouldHeartbeat(now)) {
+                // we don't need to heartbeat now, so reschedule for when we do
+                client.schedule(this, now + heartbeat.timeToNextHeartbeat(now));
+            } else {
+                heartbeat.sentHeartbeat(now);
+                RequestFuture<Void> future = sendHeartbeatRequest();
+                future.addListener(new RequestFutureListener<Void>() {
+                    @Override
+                    public void onSuccess(Void value) {
+                        long now = time.milliseconds();
+                        heartbeat.receiveHeartbeat(now);
+                        long nextHeartbeatTime = now + heartbeat.timeToNextHeartbeat(now);
+                        client.schedule(HeartbeatTask.this, nextHeartbeatTime);
+                    }
+
+                    @Override
+                    public void onFailure(RuntimeException e) {
+                        client.schedule(HeartbeatTask.this, time.milliseconds() + retryBackoffMs);
+                    }
+                });
+            }
+        }
+    }
+
+    /**
+     * Send a request to get a new partition assignment. This is a non-blocking call which sends
+     * a JoinGroup request to the coordinator (if it is available). The returned future must
+     * be polled to see if the request completed successfully.
+     * @return A request future whose completion indicates the result of the JoinGroup request.
+     */
+    private RequestFuture<Void> sendJoinGroupRequest() {
+        if (coordinatorUnknown())
+            return RequestFuture.coordinatorNotAvailable();
+
+        // send a join group request to the coordinator
+        List<String> subscribedTopics = new ArrayList<String>(subscriptions.subscribedTopics());
+        log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics);
+
+        JoinGroupRequest request = new JoinGroupRequest(groupId,
+                this.sessionTimeoutMs,
+                subscribedTopics,
+                this.consumerId,
+                this.assignmentStrategy);
+
+        // create the request for the coordinator
+        log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id());
+        return client.send(consumerCoordinator, ApiKeys.JOIN_GROUP, request)
+                .compose(new JoinGroupResponseHandler());
+    }
+
+    private class JoinGroupResponseHandler extends CoordinatorResponseHandler<JoinGroupResponse, Void> {
+
+        @Override
+        public JoinGroupResponse parse(ClientResponse response) {
+            return new JoinGroupResponse(response.responseBody());
+        }
+
+        @Override
+        public void handle(JoinGroupResponse joinResponse, RequestFuture<Void> future) {
+            // process the response
+            short errorCode = joinResponse.errorCode();
+
+            if (errorCode == Errors.NONE.code()) {
+                Coordinator.this.consumerId = joinResponse.consumerId();
+                Coordinator.this.generation = joinResponse.generationId();
+
+                // set the flag to refresh last committed offsets
+                subscriptions.needRefreshCommits();
+
+                log.debug("Joined group: {}", joinResponse.toStruct());
+
+                // record re-assignment time
+                sensors.partitionReassignments.record(response.requestLatencyMs());
+
+                // update partition assignment
+                subscriptions.changePartitionAssignment(joinResponse.assignedPartitions());
+                heartbeatTask.reset();
+                future.complete(null);
+            } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) {
+                // reset the consumer id and retry immediately
+                Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
+                log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.",
+                        groupId);
+                future.raise(Errors.UNKNOWN_CONSUMER_ID);
+            } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
+                    || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
+                // re-discover the coordinator and retry with backoff
+                coordinatorDead();
+                log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.",
+                        groupId);
+                future.raise(Errors.forCode(errorCode));
+            } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code()
+                    || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code()
+                    || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) {
+                // log the error and re-throw the exception
+                Errors error = Errors.forCode(errorCode);
+                log.error("Attempt to join group {} failed due to: {}",
+                        groupId, error.exception().getMessage());
+                future.raise(error);
+            } else {
+                // unexpected error, throw the exception
+                future.raise(new KafkaException("Unexpected error in join group response: "
+                        + Errors.forCode(joinResponse.errorCode()).exception().getMessage()));
+            }
+        }
+    }
+
+    private void commitOffsetsAsync(final Map<TopicPartition, Long> offsets, final ConsumerCommitCallback callback) {
+        this.subscriptions.needRefreshCommits();
+        RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
+        if (callback != null) {
+            future.addListener(new RequestFutureListener<Void>() {
+                @Override
+                public void onSuccess(Void value) {
+                    callback.onComplete(offsets, null);
+                }
+
+                @Override
+                public void onFailure(RuntimeException e) {
+                    callback.onComplete(offsets, e);
+                }
+            });
+        }
+    }
+
+    private void commitOffsetsSync(Map<TopicPartition, Long> offsets, ConsumerCommitCallback callback) {
+        while (true) {
+            ensureCoordinatorKnown();
+            ensurePartitionAssignment();
+
+            RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
+            client.poll(future);
+
+            if (future.succeeded()) {
+                if (callback != null)
+                    callback.onComplete(offsets, null);
+                return;
+            }
+
+            if (!future.isRetriable()) {
+                if (callback == null)
+                    throw future.exception();
+                else
+                    callback.onComplete(offsets, future.exception());
+                return;
+            }
+
+            Utils.sleep(retryBackoffMs);
+        }
+    }
+
+    /**
+     * Commit offsets for the specified list of topics and partitions. This is a non-blocking call
+     * which returns a request future that can be polled in the case of a synchronous commit or ignored in the
+     * asynchronous case.
+     *
+     * @param offsets The list of offsets per partition that should be committed.
+     * @return A request future whose value indicates whether the commit was successful or not
+     */
+    private RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, Long> offsets) {
+        if (coordinatorUnknown())
+            return RequestFuture.coordinatorNotAvailable();
+
+        if (offsets.isEmpty())
+            return RequestFuture.voidSuccess();
+
+        // create the offset commit request
+        Map<TopicPartition, OffsetCommitRequest.PartitionData> offsetData;
+        offsetData = new HashMap<TopicPartition, OffsetCommitRequest.PartitionData>(offsets.size());
+        for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet())
+            offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), ""));
+        OffsetCommitRequest req = new OffsetCommitRequest(this.groupId,
+                this.generation,
+                this.consumerId,
+                OffsetCommitRequest.DEFAULT_RETENTION_TIME,
+                offsetData);
+
+        return client.send(consumerCoordinator, ApiKeys.OFFSET_COMMIT, req)
+                .compose(new OffsetCommitResponseHandler(offsets));
+    }
+
+
+    private class OffsetCommitResponseHandler extends CoordinatorResponseHandler<OffsetCommitResponse, Void> {
+
+        private final Map<TopicPartition, Long> offsets;
+
+        public OffsetCommitResponseHandler(Map<TopicPartition, Long> offsets) {
+            this.offsets = offsets;
+        }
+
+        @Override
+        public OffsetCommitResponse parse(ClientResponse response) {
+            return new OffsetCommitResponse(response.responseBody());
+        }
+
+        @Override
+        public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> future) {
+            sensors.commitLatency.record(response.requestLatencyMs());
+            for (Map.Entry<TopicPartition, Short> entry : commitResponse.responseData().entrySet()) {
+                TopicPartition tp = entry.getKey();
+                long offset = this.offsets.get(tp);
+                short errorCode = entry.getValue();
+                if (errorCode == Errors.NONE.code()) {
+                    log.debug("Committed offset {} for partition {}", offset, tp);
+                    subscriptions.committed(tp, offset);
+                } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
+                        || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
+                    coordinatorDead();
+                    future.raise(Errors.forCode(errorCode));
+                    return;
+                } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code()
+                        || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) {
+                    // do not need to throw the exception but just log the error
+                    log.error("Error committing partition {} at offset {}: {}",
+                            tp,
+                            offset,
+                            Errors.forCode(errorCode).exception().getMessage());
+                } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
+                        || errorCode == Errors.ILLEGAL_GENERATION.code()) {
+                    // need to re-join group
+                    subscriptions.needReassignment();
+                    future.raise(Errors.forCode(errorCode));
+                    return;
+                } else {
+                    // do not need to throw the exception but just log the error
+                    future.raise(Errors.forCode(errorCode));
+                    log.error("Error committing partition {} at offset {}: {}",
+                            tp,
+                            offset,
+                            Errors.forCode(errorCode).exception().getMessage());
+                }
+            }
+
+            future.complete(null);
+        }
+    }
+
+    /**
+     * Fetch the committed offsets for a set of partitions. This is a non-blocking call. The
+     * returned future can be polled to get the actual offsets returned from the broker.
+     *
+     * @param partitions The set of partitions to get offsets for.
+     * @return A request future containing the committed offsets.
+     */
+    private RequestFuture<Map<TopicPartition, Long>> sendOffsetFetchRequest(Set<TopicPartition> partitions) {
+        if (coordinatorUnknown())
+            return RequestFuture.coordinatorNotAvailable();
+
+        log.debug("Fetching committed offsets for partitions: {}",  Utils.join(partitions, ", "));
+        // construct the request
+        OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList<TopicPartition>(partitions));
+
+        // send the request with a callback
+        return client.send(consumerCoordinator, ApiKeys.OFFSET_FETCH, request)
+                .compose(new OffsetFetchResponseHandler());
+    }
+
+    private class OffsetFetchResponseHandler extends CoordinatorResponseHandler<OffsetFetchResponse, Map<TopicPartition, Long>> {
+
+        @Override
+        public OffsetFetchResponse parse(ClientResponse response) {
+            return new OffsetFetchResponse(response.responseBody());
+        }
+
+        @Override
+        public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, Long>> future) {
+            Map<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>(response.responseData().size());
+            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) {
+                TopicPartition tp = entry.getKey();
+                OffsetFetchResponse.PartitionData data = entry.getValue();
+                if (data.hasError()) {
+                    log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode)
+                            .exception()
+                            .getMessage());
+                    if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) {
+                        // just retry
+                        future.raise(Errors.OFFSET_LOAD_IN_PROGRESS);
+                    } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
+                        // re-discover the coordinator and retry
+                        coordinatorDead();
+                        future.raise(Errors.NOT_COORDINATOR_FOR_CONSUMER);
+                    } else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
+                            || data.errorCode == Errors.ILLEGAL_GENERATION.code()) {
+                        // need to re-join group
+                        subscriptions.needReassignment();
+                        future.raise(Errors.forCode(data.errorCode));
+                    } else {
+                        future.raise(new KafkaException("Unexpected error in fetch offset response: "
+                                + Errors.forCode(data.errorCode).exception().getMessage()));
+                    }
+                    return;
+                } else if (data.offset >= 0) {
+                    // record the position with the offset (-1 indicates no committed offset to fetch)
+                    offsets.put(tp, data.offset);
+                } else {
+                    log.debug("No committed offset for partition " + tp);
+                }
+            }
+
+            future.complete(offsets);
+        }
+    }
+
+    /**
+     * Send a heartbeat request now (visible only for testing).
+     */
+    public RequestFuture<Void> sendHeartbeatRequest() {
+        HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId);
+        return client.send(consumerCoordinator, ApiKeys.HEARTBEAT, req)
+                .compose(new HeartbeatCompletionHandler());
+    }
+
+    public boolean coordinatorUnknown() {
+        return this.consumerCoordinator == null;
+    }
+
+    /**
+     * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to
+     * one of the brokers. The returned future should be polled to get the result of the request.
+     * @return A request future which indicates the completion of the metadata request
+     */
+    private RequestFuture<Void> sendConsumerMetadataRequest() {
+        // initiate the consumer metadata request
+        // find a node to ask about the coordinator
+        Node node = this.client.leastLoadedNode();
+        if (node == null) {
+            // TODO: If there are no brokers left, perhaps we should use the bootstrap set
+            // from configuration?
+            return RequestFuture.noBrokersAvailable();
+        } else {
+            // create a consumer metadata request
+            log.debug("Issuing consumer metadata request to broker {}", node.id());
+            ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId);
+            return client.send(node, ApiKeys.CONSUMER_METADATA, metadataRequest)
+                    .compose(new RequestFutureAdapter<ClientResponse, Void>() {
+                        @Override
+                        public void onSuccess(ClientResponse response, RequestFuture<Void> future) {
+                            handleConsumerMetadataResponse(response, future);
+                        }
+                    });
+        }
+    }
+
+    private void handleConsumerMetadataResponse(ClientResponse resp, RequestFuture<Void> future) {
+        log.debug("Consumer metadata response {}", resp);
+
+        // parse the response to get the coordinator info if it is not disconnected,
+        // otherwise we need to request metadata update
+        if (resp.wasDisconnected()) {
+            future.raise(new DisconnectException());
+        } else if (!coordinatorUnknown()) {
+            // We already found the coordinator, so ignore the request
+            future.complete(null);
+        } else {
+            ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody());
+            // use MAX_VALUE - node.id as the coordinator id to mimic separate connections
+            // for the coordinator in the underlying network client layer
+            // TODO: this needs to be better handled in KAFKA-1935
+            if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) {
+                this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(),
+                        consumerMetadataResponse.node().host(),
+                        consumerMetadataResponse.node().port());
+                heartbeatTask.reset();
+                future.complete(null);
+            } else {
+                future.raise(Errors.forCode(consumerMetadataResponse.errorCode()));
+            }
+        }
+    }
+
+    /**
+     * Mark the current coordinator as dead.
+     */
+    private void coordinatorDead() {
+        if (this.consumerCoordinator != null) {
+            log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id());
+            this.consumerCoordinator = null;
+        }
+    }
+
+    private class HeartbeatCompletionHandler extends CoordinatorResponseHandler<HeartbeatResponse, Void> {
+        @Override
+        public HeartbeatResponse parse(ClientResponse response) {
+            return new HeartbeatResponse(response.responseBody());
+        }
+
+        @Override
+        public void handle(HeartbeatResponse heartbeatResponse, RequestFuture<Void> future) {
+            sensors.heartbeatLatency.record(response.requestLatencyMs());
+            short error = heartbeatResponse.errorCode();
+            if (error == Errors.NONE.code()) {
+                log.debug("Received successful heartbeat response.");
+                future.complete(null);
+            } else if (error == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
+                    || error == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
+                log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead.");
+                coordinatorDead();
+                future.raise(Errors.forCode(error));
+            } else if (error == Errors.ILLEGAL_GENERATION.code()) {
+                log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group.");
+                subscriptions.needReassignment();
+                future.raise(Errors.ILLEGAL_GENERATION);
+            } else if (error == Errors.UNKNOWN_CONSUMER_ID.code()) {
+                log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group.");
+                consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
+                subscriptions.needReassignment();
+                future.raise(Errors.UNKNOWN_CONSUMER_ID);
+            } else {
+                future.raise(new KafkaException("Unexpected error in heartbeat response: "
+                        + Errors.forCode(error).exception().getMessage()));
+            }
+        }
+    }
+
+    private abstract class CoordinatorResponseHandler<R, T>
+            extends RequestFutureAdapter<ClientResponse, T> {
+        protected ClientResponse response;
+
+        public abstract R parse(ClientResponse response);
+
+        public abstract void handle(R response, RequestFuture<T> future);
+
+        @Override
+        public void onSuccess(ClientResponse clientResponse, RequestFuture<T> future) {
+            this.response = clientResponse;
+
+            if (clientResponse.wasDisconnected()) {
+                int correlation = response.request().request().header().correlationId();
+                log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected",
+                        response.request(),
+                        correlation,
+                        response.request().request().destination());
+
+                // mark the coordinator as dead
+                coordinatorDead();
+                future.raise(new DisconnectException());
+                return;
+            }
+
+            R response = parse(clientResponse);
+            handle(response, future);
+        }
+
+        @Override
+        public void onFailure(RuntimeException e, RequestFuture<T> future) {
+            if (e instanceof DisconnectException) {
+                log.debug("Coordinator request failed", e);
+                coordinatorDead();
+            }
+            future.raise(e);
+        }
+    }
+
+    public interface RebalanceCallback {
+        void onPartitionsAssigned(Collection<TopicPartition> partitions);
+        void onPartitionsRevoked(Collection<TopicPartition> partitions);
+    }
+
+    private class CoordinatorMetrics {
+        public final Metrics metrics;
+        public final String metricGrpName;
+
+        public final Sensor commitLatency;
+        public final Sensor heartbeatLatency;
+        public final Sensor partitionReassignments;
+
+        public CoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
+            this.metrics = metrics;
+            this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
+
+            this.commitLatency = metrics.sensor("commit-latency");
+            this.commitLatency.add(new MetricName("commit-latency-avg",
+                this.metricGrpName,
+                "The average time taken for a commit request",
+                tags), new Avg());
+            this.commitLatency.add(new MetricName("commit-latency-max",
+                this.metricGrpName,
+                "The max time taken for a commit request",
+                tags), new Max());
+            this.commitLatency.add(new MetricName("commit-rate",
+                this.metricGrpName,
+                "The number of commit calls per second",
+                tags), new Rate(new Count()));
+
+            this.heartbeatLatency = metrics.sensor("heartbeat-latency");
+            this.heartbeatLatency.add(new MetricName("heartbeat-response-time-max",
+                this.metricGrpName,
+                "The max time taken to receive a response to a hearbeat request",
+                tags), new Max());
+            this.heartbeatLatency.add(new MetricName("heartbeat-rate",
+                this.metricGrpName,
+                "The average number of heartbeats per second",
+                tags), new Rate(new Count()));
+
+            this.partitionReassignments = metrics.sensor("reassignment-latency");
+            this.partitionReassignments.add(new MetricName("reassignment-time-avg",
+                this.metricGrpName,
+                "The average time taken for a partition reassignment",
+                tags), new Avg());
+            this.partitionReassignments.add(new MetricName("reassignment-time-max",
+                this.metricGrpName,
+                "The max time taken for a partition reassignment",
+                tags), new Avg());
+            this.partitionReassignments.add(new MetricName("reassignment-rate",
+                this.metricGrpName,
+                "The number of partition reassignments per second",
+                tags), new Rate(new Count()));
+
+            Measurable numParts =
+                new Measurable() {
+                    public double measure(MetricConfig config, long now) {
+                        return subscriptions.assignedPartitions().size();
+                    }
+                };
+            metrics.addMetric(new MetricName("assigned-partitions",
+                this.metricGrpName,
+                "The number of partitions currently assigned to this consumer",
+                tags),
+                numParts);
+
+            Measurable lastHeartbeat =
+                new Measurable() {
+                    public double measure(MetricConfig config, long now) {
+                        return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS);
+                    }
+                };
+            metrics.addMetric(new MetricName("last-heartbeat-seconds-ago",
+                this.metricGrpName,
+                "The number of seconds since the last controller heartbeat",
+                tags),
+                lastHeartbeat);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTask.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTask.java
new file mode 100644
index 0000000..7792dff
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTask.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package org.apache.flink.kafka_backport.clients.consumer.internals;
+
+
+public interface DelayedTask {
+
+    /**
+     * Execute the task.
+     * @param now current time in milliseconds
+     */
+    void run(long now);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTaskQueue.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTaskQueue.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTaskQueue.java
new file mode 100644
index 0000000..fbd1e19
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTaskQueue.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.kafka_backport.clients.consumer.internals;
+
+import java.util.Iterator;
+import java.util.PriorityQueue;
+
+/**
+ * Tracks a set of tasks to be executed after a delay.
+ */
+public class DelayedTaskQueue {
+
+    private PriorityQueue<Entry> tasks;
+
+    public DelayedTaskQueue() {
+        tasks = new PriorityQueue<Entry>();
+    }
+
+    /**
+     * Schedule a task for execution in the future.
+     *
+     * @param task the task to execute
+     * @param at the time at which to
+     */
+    public void add(DelayedTask task, long at) {
+        tasks.add(new Entry(task, at));
+    }
+
+    /**
+     * Remove a task from the queue if it is present
+     * @param task the task to be removed
+     * @returns true if a task was removed as a result of this call
+     */
+    public boolean remove(DelayedTask task) {
+        boolean wasRemoved = false;
+        Iterator<Entry> iterator = tasks.iterator();
+        while (iterator.hasNext()) {
+            Entry entry = iterator.next();
+            if (entry.task.equals(task)) {
+                iterator.remove();
+                wasRemoved = true;
+            }
+        }
+        return wasRemoved;
+    }
+
+    /**
+     * Get amount of time in milliseconds until the next event. Returns Long.MAX_VALUE if no tasks are scheduled.
+     *
+     * @return the remaining time in milliseconds
+     */
+    public long nextTimeout(long now) {
+        if (tasks.isEmpty())
+            return Long.MAX_VALUE;
+        else
+            return Math.max(tasks.peek().timeout - now, 0);
+    }
+
+    /**
+     * Run any ready tasks.
+     *
+     * @param now the current time
+     */
+    public void poll(long now) {
+        while (!tasks.isEmpty() && tasks.peek().timeout <= now) {
+            Entry entry = tasks.poll();
+            entry.task.run(now);
+        }
+    }
+
+    private static class Entry implements Comparable<Entry> {
+        DelayedTask task;
+        long timeout;
+
+        public Entry(DelayedTask task, long timeout) {
+            this.task = task;
+            this.timeout = timeout;
+        }
+
+        @Override
+        public int compareTo(Entry entry) {
+            return Long.compare(timeout, entry.timeout);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Fetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Fetcher.java
new file mode 100644
index 0000000..6962a54
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Fetcher.java
@@ -0,0 +1,506 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package org.apache.flink.kafka_backport.clients.consumer.internals;
+
+import org.apache.flink.kafka_backport.clients.ClientResponse;
+import org.apache.flink.kafka_backport.clients.Metadata;
+import org.apache.flink.kafka_backport.clients.consumer.NoOffsetForPartitionException;
+import org.apache.flink.kafka_backport.clients.consumer.OffsetResetStrategy;
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.PartitionInfo;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.errors.InvalidMetadataException;
+import org.apache.flink.kafka_backport.common.metrics.Metrics;
+import org.apache.flink.kafka_backport.common.metrics.Sensor;
+import org.apache.flink.kafka_backport.common.metrics.stats.Avg;
+import org.apache.flink.kafka_backport.common.metrics.stats.Rate;
+import org.apache.flink.kafka_backport.common.record.LogEntry;
+import org.apache.flink.kafka_backport.common.record.MemoryRecords;
+import org.apache.flink.kafka_backport.common.requests.FetchRequest;
+import org.apache.flink.kafka_backport.common.requests.FetchResponse;
+import org.apache.flink.kafka_backport.common.serialization.Deserializer;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+import org.apache.flink.kafka_backport.clients.consumer.ConsumerRecord;
+import org.apache.flink.kafka_backport.common.Cluster;
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.apache.flink.kafka_backport.common.errors.DisconnectException;
+import org.apache.flink.kafka_backport.common.metrics.stats.Count;
+import org.apache.flink.kafka_backport.common.metrics.stats.Max;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+import org.apache.flink.kafka_backport.common.requests.ListOffsetRequest;
+import org.apache.flink.kafka_backport.common.requests.ListOffsetResponse;
+import org.apache.flink.kafka_backport.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This class manage the fetching process with the brokers.
+ */
+public class Fetcher<K, V> {
+    private static final long EARLIEST_OFFSET_TIMESTAMP = -2L;
+    private static final long LATEST_OFFSET_TIMESTAMP = -1L;
+
+    private static final Logger log = LoggerFactory.getLogger(Fetcher.class);
+
+    private final ConsumerNetworkClient client;
+    private final Time time;
+    private final int minBytes;
+    private final int maxWaitMs;
+    private final int fetchSize;
+    private final long retryBackoffMs;
+    private final boolean checkCrcs;
+    private final Metadata metadata;
+    private final FetchManagerMetrics sensors;
+    private final SubscriptionState subscriptions;
+    private final List<PartitionRecords<K, V>> records;
+    private final Deserializer<K> keyDeserializer;
+    private final Deserializer<V> valueDeserializer;
+
+    public Fetcher(ConsumerNetworkClient client,
+                   int minBytes,
+                   int maxWaitMs,
+                   int fetchSize,
+                   boolean checkCrcs,
+                   Deserializer<K> keyDeserializer,
+                   Deserializer<V> valueDeserializer,
+                   Metadata metadata,
+                   SubscriptionState subscriptions,
+                   Metrics metrics,
+                   String metricGrpPrefix,
+                   Map<String, String> metricTags,
+                   Time time,
+                   long retryBackoffMs) {
+
+        this.time = time;
+        this.client = client;
+        this.metadata = metadata;
+        this.subscriptions = subscriptions;
+        this.minBytes = minBytes;
+        this.maxWaitMs = maxWaitMs;
+        this.fetchSize = fetchSize;
+        this.checkCrcs = checkCrcs;
+
+        this.keyDeserializer = keyDeserializer;
+        this.valueDeserializer = valueDeserializer;
+
+        this.records = new LinkedList<PartitionRecords<K, V>>();
+
+        this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags);
+        this.retryBackoffMs = retryBackoffMs;
+    }
+
+    /**
+     * Set-up a fetch request for any node that we have assigned partitions for which doesn't have one.
+     *
+     * @param cluster The current cluster metadata
+     */
+    public void initFetches(Cluster cluster) {
+        for (Map.Entry<Node, FetchRequest> fetchEntry: createFetchRequests(cluster).entrySet()) {
+            final FetchRequest fetch = fetchEntry.getValue();
+            client.send(fetchEntry.getKey(), ApiKeys.FETCH, fetch)
+                    .addListener(new RequestFutureListener<ClientResponse>() {
+                        @Override
+                        public void onSuccess(ClientResponse response) {
+                            handleFetchResponse(response, fetch);
+                        }
+
+                        @Override
+                        public void onFailure(RuntimeException e) {
+                            log.debug("Fetch failed", e);
+                        }
+                    });
+        }
+    }
+
+    /**
+     * Update the fetch positions for the provided partitions.
+     * @param partitions
+     */
+    public void updateFetchPositions(Set<TopicPartition> partitions) {
+        // reset the fetch position to the committed position
+        for (TopicPartition tp : partitions) {
+            // skip if we already have a fetch position
+            if (subscriptions.fetched(tp) != null)
+                continue;
+
+            // TODO: If there are several offsets to reset, we could submit offset requests in parallel
+            if (subscriptions.isOffsetResetNeeded(tp)) {
+                resetOffset(tp);
+            } else if (subscriptions.committed(tp) == null) {
+                // there's no committed position, so we need to reset with the default strategy
+                subscriptions.needOffsetReset(tp);
+                resetOffset(tp);
+            } else {
+                log.debug("Resetting offset for partition {} to the committed offset {}",
+                        tp, subscriptions.committed(tp));
+                subscriptions.seek(tp, subscriptions.committed(tp));
+            }
+        }
+    }
+
+    /**
+     * Reset offsets for the given partition using the offset reset strategy.
+     *
+     * @param partition The given partition that needs reset offset
+     * @throws org.apache.flink.kafka_backport.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined
+     */
+    private void resetOffset(TopicPartition partition) {
+        OffsetResetStrategy strategy = subscriptions.resetStrategy(partition);
+        final long timestamp;
+        if (strategy == OffsetResetStrategy.EARLIEST)
+            timestamp = EARLIEST_OFFSET_TIMESTAMP;
+        else if (strategy == OffsetResetStrategy.LATEST)
+            timestamp = LATEST_OFFSET_TIMESTAMP;
+        else
+            throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined");
+
+        log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase());
+        long offset = listOffset(partition, timestamp);
+        this.subscriptions.seek(partition, offset);
+    }
+
+    /**
+     * Fetch a single offset before the given timestamp for the partition.
+     *
+     * @param partition The partition that needs fetching offset.
+     * @param timestamp The timestamp for fetching offset.
+     * @return The offset of the message that is published before the given timestamp
+     */
+    private long listOffset(TopicPartition partition, long timestamp) {
+        while (true) {
+            RequestFuture<Long> future = sendListOffsetRequest(partition, timestamp);
+            client.poll(future);
+
+            if (future.succeeded())
+                return future.value();
+
+            if (!future.isRetriable())
+                throw future.exception();
+
+            if (future.exception() instanceof InvalidMetadataException)
+                client.awaitMetadataUpdate();
+            else
+                Utils.sleep(retryBackoffMs);
+        }
+    }
+
+    /**
+     * Return the fetched records, empty the record buffer and update the consumed position.
+     *
+     * @return The fetched records per partition
+     */
+    public Map<TopicPartition, List<ConsumerRecord<K, V>>> fetchedRecords() {
+        if (this.subscriptions.partitionAssignmentNeeded()) {
+            return Collections.emptyMap();
+        } else {
+            Map<TopicPartition, List<ConsumerRecord<K, V>>> drained = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
+            for (PartitionRecords<K, V> part : this.records) {
+                Long consumed = subscriptions.consumed(part.partition);
+                if (this.subscriptions.assignedPartitions().contains(part.partition)
+                    && (consumed == null || part.fetchOffset == consumed)) {
+                    List<ConsumerRecord<K, V>> records = drained.get(part.partition);
+                    if (records == null) {
+                        records = part.records;
+                        drained.put(part.partition, records);
+                    } else {
+                        records.addAll(part.records);
+                    }
+                    subscriptions.consumed(part.partition, part.records.get(part.records.size() - 1).offset() + 1);
+                } else {
+                    // these records aren't next in line based on the last consumed position, ignore them
+                    // they must be from an obsolete request
+                    log.debug("Ignoring fetched records for {} at offset {}", part.partition, part.fetchOffset);
+                }
+            }
+            this.records.clear();
+            return drained;
+        }
+    }
+
+    /**
+     * Fetch a single offset before the given timestamp for the partition.
+     *
+     * @param topicPartition The partition that needs fetching offset.
+     * @param timestamp The timestamp for fetching offset.
+     * @return A response which can be polled to obtain the corresponding offset.
+     */
+    private RequestFuture<Long> sendListOffsetRequest(final TopicPartition topicPartition, long timestamp) {
+        Map<TopicPartition, ListOffsetRequest.PartitionData> partitions = new HashMap<TopicPartition, ListOffsetRequest.PartitionData>(1);
+        partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1));
+        PartitionInfo info = metadata.fetch().partition(topicPartition);
+        if (info == null) {
+            metadata.add(topicPartition.topic());
+            log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition);
+            return RequestFuture.staleMetadata();
+        } else if (info.leader() == null) {
+            log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition);
+            return RequestFuture.leaderNotAvailable();
+        } else {
+            Node node = info.leader();
+            ListOffsetRequest request = new ListOffsetRequest(-1, partitions);
+            return client.send(node, ApiKeys.LIST_OFFSETS, request)
+                    .compose(new RequestFutureAdapter<ClientResponse, Long>() {
+                        @Override
+                        public void onSuccess(ClientResponse response, RequestFuture<Long> future) {
+                            handleListOffsetResponse(topicPartition, response, future);
+                        }
+                    });
+        }
+    }
+
+    /**
+     * Callback for the response of the list offset call above.
+     * @param topicPartition The partition that was fetched
+     * @param clientResponse The response from the server.
+     */
+    private void handleListOffsetResponse(TopicPartition topicPartition,
+                                          ClientResponse clientResponse,
+                                          RequestFuture<Long> future) {
+        if (clientResponse.wasDisconnected()) {
+            future.raise(new DisconnectException());
+        } else {
+            ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody());
+            short errorCode = lor.responseData().get(topicPartition).errorCode;
+            if (errorCode == Errors.NONE.code()) {
+                List<Long> offsets = lor.responseData().get(topicPartition).offsets;
+                if (offsets.size() != 1)
+                    throw new IllegalStateException("This should not happen.");
+                long offset = offsets.get(0);
+                log.debug("Fetched offset {} for partition {}", offset, topicPartition);
+
+                future.complete(offset);
+            } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code()
+                    || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) {
+                log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.",
+                        topicPartition);
+                future.raise(Errors.forCode(errorCode));
+            } else {
+                log.error("Attempt to fetch offsets for partition {} failed due to: {}",
+                        topicPartition, Errors.forCode(errorCode).exception().getMessage());
+                future.raise(new StaleMetadataException());
+            }
+        }
+    }
+
+    /**
+     * Create fetch requests for all nodes for which we have assigned partitions
+     * that have no existing requests in flight.
+     */
+    private Map<Node, FetchRequest> createFetchRequests(Cluster cluster) {
+        // create the fetch info
+        Map<Node, Map<TopicPartition, FetchRequest.PartitionData>> fetchable = new HashMap<Node, Map<TopicPartition, FetchRequest.PartitionData>>();
+        for (TopicPartition partition : subscriptions.assignedPartitions()) {
+            Node node = cluster.leaderFor(partition);
+            if (node == null) {
+                metadata.requestUpdate();
+            } else if (this.client.pendingRequestCount(node) == 0) {
+                // if there is a leader and no in-flight requests, issue a new fetch
+                Map<TopicPartition, FetchRequest.PartitionData> fetch = fetchable.get(node);
+                if (fetch == null) {
+                    fetch = new HashMap<TopicPartition, FetchRequest.PartitionData>();
+                    fetchable.put(node, fetch);
+                }
+                long offset = this.subscriptions.fetched(partition);
+                fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize));
+            }
+        }
+
+        // create the fetches
+        Map<Node, FetchRequest> requests = new HashMap<Node, FetchRequest>();
+        for (Map.Entry<Node, Map<TopicPartition, FetchRequest.PartitionData>> entry : fetchable.entrySet()) {
+            Node node = entry.getKey();
+            FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue());
+            requests.put(node, fetch);
+        }
+        return requests;
+    }
+
+    /**
+     * The callback for fetch completion
+     */
+    private void handleFetchResponse(ClientResponse resp, FetchRequest request) {
+        if (resp.wasDisconnected()) {
+            int correlation = resp.request().request().header().correlationId();
+            log.debug("Cancelled fetch request {} with correlation id {} due to node {} being disconnected",
+                resp.request(), correlation, resp.request().request().destination());
+        } else {
+            int totalBytes = 0;
+            int totalCount = 0;
+            FetchResponse response = new FetchResponse(resp.responseBody());
+            for (Map.Entry<TopicPartition, FetchResponse.PartitionData> entry : response.responseData().entrySet()) {
+                TopicPartition tp = entry.getKey();
+                FetchResponse.PartitionData partition = entry.getValue();
+                if (!subscriptions.assignedPartitions().contains(tp)) {
+                    log.debug("Ignoring fetched data for partition {} which is no longer assigned.", tp);
+                } else if (partition.errorCode == Errors.NONE.code()) {
+                    int bytes = 0;
+                    ByteBuffer buffer = partition.recordSet;
+                    MemoryRecords records = MemoryRecords.readableRecords(buffer);
+                    long fetchOffset = request.fetchData().get(tp).offset;
+                    List<ConsumerRecord<K, V>> parsed = new ArrayList<ConsumerRecord<K, V>>();
+                    for (LogEntry logEntry : records) {
+                        parsed.add(parseRecord(tp, logEntry));
+                        bytes += logEntry.size();
+                    }
+                    if (parsed.size() > 0) {
+                        ConsumerRecord<K, V> record = parsed.get(parsed.size() - 1);
+                        this.subscriptions.fetched(tp, record.offset() + 1);
+                        this.records.add(new PartitionRecords<K, V>(fetchOffset, tp, parsed));
+                        this.sensors.recordsFetchLag.record(partition.highWatermark - record.offset());
+                    }
+                    this.sensors.recordTopicFetchMetrics(tp.topic(), bytes, parsed.size());
+                    totalBytes += bytes;
+                    totalCount += parsed.size();
+                } else if (partition.errorCode == Errors.NOT_LEADER_FOR_PARTITION.code()
+                    || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) {
+                    this.metadata.requestUpdate();
+                } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) {
+                    // TODO: this could be optimized by grouping all out-of-range partitions
+                    log.info("Fetch offset {} is out of range, resetting offset", subscriptions.fetched(tp));
+                    subscriptions.needOffsetReset(tp);
+                } else if (partition.errorCode == Errors.UNKNOWN.code()) {
+                    log.warn("Unknown error fetching data for topic-partition {}", tp);
+                } else {
+                    throw new IllegalStateException("Unexpected error code " + partition.errorCode + " while fetching data");
+                }
+            }
+            this.sensors.bytesFetched.record(totalBytes);
+            this.sensors.recordsFetched.record(totalCount);
+        }
+        this.sensors.fetchLatency.record(resp.requestLatencyMs());
+    }
+
+    /**
+     * Parse the record entry, deserializing the key / value fields if necessary
+     */
+    private ConsumerRecord<K, V> parseRecord(TopicPartition partition, LogEntry logEntry) {
+        if (this.checkCrcs)
+            logEntry.record().ensureValid();
+
+        long offset = logEntry.offset();
+        ByteBuffer keyBytes = logEntry.record().key();
+        K key = keyBytes == null ? null : this.keyDeserializer.deserialize(partition.topic(), Utils.toArray(keyBytes));
+        ByteBuffer valueBytes = logEntry.record().value();
+        V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), Utils.toArray(valueBytes));
+
+        return new ConsumerRecord<K, V>(partition.topic(), partition.partition(), offset, key, value);
+    }
+
+    private static class PartitionRecords<K, V> {
+        public long fetchOffset;
+        public TopicPartition partition;
+        public List<ConsumerRecord<K, V>> records;
+
+        public PartitionRecords(long fetchOffset, TopicPartition partition, List<ConsumerRecord<K, V>> records) {
+            this.fetchOffset = fetchOffset;
+            this.partition = partition;
+            this.records = records;
+        }
+    }
+
+    private class FetchManagerMetrics {
+        public final Metrics metrics;
+        public final String metricGrpName;
+
+        public final Sensor bytesFetched;
+        public final Sensor recordsFetched;
+        public final Sensor fetchLatency;
+        public final Sensor recordsFetchLag;
+
+
+        public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
+            this.metrics = metrics;
+            this.metricGrpName = metricGrpPrefix + "-fetch-manager-metrics";
+
+            this.bytesFetched = metrics.sensor("bytes-fetched");
+            this.bytesFetched.add(new MetricName("fetch-size-avg",
+                this.metricGrpName,
+                "The average number of bytes fetched per request",
+                tags), new Avg());
+            this.bytesFetched.add(new MetricName("fetch-size-max",
+                this.metricGrpName,
+                "The maximum number of bytes fetched per request",
+                tags), new Max());
+            this.bytesFetched.add(new MetricName("bytes-consumed-rate",
+                this.metricGrpName,
+                "The average number of bytes consumed per second",
+                tags), new Rate());
+
+            this.recordsFetched = metrics.sensor("records-fetched");
+            this.recordsFetched.add(new MetricName("records-per-request-avg",
+                this.metricGrpName,
+                "The average number of records in each request",
+                tags), new Avg());
+            this.recordsFetched.add(new MetricName("records-consumed-rate",
+                this.metricGrpName,
+                "The average number of records consumed per second",
+                tags), new Rate());
+
+            this.fetchLatency = metrics.sensor("fetch-latency");
+            this.fetchLatency.add(new MetricName("fetch-latency-avg",
+                this.metricGrpName,
+                "The average time taken for a fetch request.",
+                tags), new Avg());
+            this.fetchLatency.add(new MetricName("fetch-latency-max",
+                this.metricGrpName,
+                "The max time taken for any fetch request.",
+                tags), new Max());
+            this.fetchLatency.add(new MetricName("fetch-rate",
+                this.metricGrpName,
+                "The number of fetch requests per second.",
+                tags), new Rate(new Count()));
+
+            this.recordsFetchLag = metrics.sensor("records-lag");
+            this.recordsFetchLag.add(new MetricName("records-lag-max",
+                this.metricGrpName,
+                "The maximum lag in terms of number of records for any partition in this window",
+                tags), new Max());
+        }
+
+        public void recordTopicFetchMetrics(String topic, int bytes, int records) {
+            // record bytes fetched
+            String name = "topic." + topic + ".bytes-fetched";
+            Sensor bytesFetched = this.metrics.getSensor(name);
+            if (bytesFetched == null)
+                bytesFetched = this.metrics.sensor(name);
+            bytesFetched.record(bytes);
+
+            // record records fetched
+            name = "topic." + topic + ".records-fetched";
+            Sensor recordsFetched = this.metrics.getSensor(name);
+            if (recordsFetched == null)
+                recordsFetched = this.metrics.sensor(name);
+            recordsFetched.record(records);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Heartbeat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Heartbeat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Heartbeat.java
new file mode 100644
index 0000000..f412897
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Heartbeat.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.kafka_backport.clients.consumer.internals;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A helper class for managing the heartbeat to the coordinator
+ */
+public final class Heartbeat {
+    
+    /* The number of heartbeats to attempt to complete per session timeout interval.
+     * so, e.g., with a session timeout of 3 seconds we would attempt a heartbeat
+     * once per second.
+     */
+    public final static int HEARTBEATS_PER_SESSION_INTERVAL = 3;
+
+    private final long timeout;
+    private long lastHeartbeatSend;
+    private long lastHeartbeatReceive;
+    private long lastSessionReset;
+
+    public Heartbeat(long timeout, long now) {
+        this.timeout = timeout;
+        this.lastSessionReset = now;
+    }
+
+    public void sentHeartbeat(long now) {
+        this.lastHeartbeatSend = now;
+    }
+
+    public void receiveHeartbeat(long now) {
+        this.lastHeartbeatReceive = now;
+    }
+
+    public boolean shouldHeartbeat(long now) {
+        return timeToNextHeartbeat(now) == 0;
+    }
+    
+    public long lastHeartbeatSend() {
+        return this.lastHeartbeatSend;
+    }
+
+    public long timeToNextHeartbeat(long now) {
+        long timeSinceLastHeartbeat = now - Math.max(lastHeartbeatSend, lastSessionReset);
+
+        long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL;
+        if (timeSinceLastHeartbeat > hbInterval)
+            return 0;
+        else
+            return hbInterval - timeSinceLastHeartbeat;
+    }
+
+    public boolean sessionTimeoutExpired(long now) {
+        return now - Math.max(lastSessionReset, lastHeartbeatReceive) > timeout;
+    }
+
+    public long interval() {
+        return timeout / HEARTBEATS_PER_SESSION_INTERVAL;
+    }
+
+    public void resetSessionTimeout(long now) {
+        this.lastSessionReset = now;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoAvailableBrokersException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoAvailableBrokersException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoAvailableBrokersException.java
new file mode 100644
index 0000000..421c64e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoAvailableBrokersException.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.kafka_backport.clients.consumer.internals;
+
+import org.apache.flink.kafka_backport.common.errors.InvalidMetadataException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * No brokers were available to complete a request.
+ */
+public class NoAvailableBrokersException extends InvalidMetadataException {
+    private static final long serialVersionUID = 1L;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoOpConsumerRebalanceCallback.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
new file mode 100644
index 0000000..6da31dd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoOpConsumerRebalanceCallback.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.kafka_backport.clients.consumer.internals;
+
+import org.apache.flink.kafka_backport.clients.consumer.ConsumerRebalanceCallback;
+import org.apache.flink.kafka_backport.clients.consumer.Consumer;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+
+import java.util.Collection;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class NoOpConsumerRebalanceCallback implements ConsumerRebalanceCallback {
+
+    @Override
+    public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {}
+
+    @Override
+    public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFuture.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFuture.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFuture.java
new file mode 100644
index 0000000..7b6edc3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFuture.java
@@ -0,0 +1,211 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer.internals;
+
+import org.apache.flink.kafka_backport.common.errors.RetriableException;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+
+import java.util.ArrayList;
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Result of an asynchronous request from {@link ConsumerNetworkClient}. Use {@link ConsumerNetworkClient#poll(long)}
+ * (and variants) to finish a request future. Use {@link #isDone()} to check if the future is complete, and
+ * {@link #succeeded()} to check if the request completed successfully. Typical usage might look like this:
+ *
+ * <pre>
+ *     RequestFuture<ClientResponse> future = client.send(api, request);
+ *     client.poll(future);
+ *
+ *     if (future.succeeded()) {
+ *         ClientResponse response = future.value();
+ *         // Handle response
+ *     } else {
+ *         throw future.exception();
+ *     }
+ * </pre>
+ *
+ * @param <T> Return type of the result (Can be Void if there is no response)
+ */
+public class RequestFuture<T> {
+
+    private boolean isDone = false;
+    private T value;
+    private RuntimeException exception;
+    private List<RequestFutureListener<T>> listeners = new ArrayList<RequestFutureListener<T>>();
+
+
+    /**
+     * Check whether the response is ready to be handled
+     * @return true if the response is ready, false otherwise
+     */
+    public boolean isDone() {
+        return isDone;
+    }
+
+    /**
+     * Get the value corresponding to this request (only available if the request succeeded)
+     * @return the value if it exists or null
+     */
+    public T value() {
+        return value;
+    }
+
+    /**
+     * Check if the request succeeded;
+     * @return true if the request completed and was successful
+     */
+    public boolean succeeded() {
+        return isDone && exception == null;
+    }
+
+    /**
+     * Check if the request failed.
+     * @return true if the request completed with a failure
+     */
+    public boolean failed() {
+        return isDone && exception != null;
+    }
+
+    /**
+     * Check if the request is retriable (convenience method for checking if
+     * the exception is an instance of {@link org.apache.flink.kafka_backport.common.errors.RetriableException}.
+     * @return true if it is retriable, false otherwise
+     */
+    public boolean isRetriable() {
+        return exception instanceof RetriableException;
+    }
+
+    /**
+     * Get the exception from a failed result (only available if the request failed)
+     * @return The exception if it exists or null
+     */
+    public RuntimeException exception() {
+        return exception;
+    }
+
+    /**
+     * Complete the request successfully. After this call, {@link #succeeded()} will return true
+     * and the value can be obtained through {@link #value()}.
+     * @param value corresponding value (or null if there is none)
+     */
+    public void complete(T value) {
+        this.value = value;
+        this.isDone = true;
+        fireSuccess();
+    }
+
+    /**
+     * Raise an exception. The request will be marked as failed, and the caller can either
+     * handle the exception or throw it.
+     * @param e corresponding exception to be passed to caller
+     */
+    public void raise(RuntimeException e) {
+        this.exception = e;
+        this.isDone = true;
+        fireFailure();
+    }
+
+    /**
+     * Raise an error. The request will be marked as failed.
+     * @param error corresponding error to be passed to caller
+     */
+    public void raise(Errors error) {
+        raise(error.exception());
+    }
+
+    private void fireSuccess() {
+        for (RequestFutureListener listener: listeners)
+            listener.onSuccess(value);
+    }
+
+    private void fireFailure() {
+        for (RequestFutureListener listener: listeners)
+            listener.onFailure(exception);
+    }
+
+    /**
+     * Add a listener which will be notified when the future completes
+     * @param listener
+     */
+    public void addListener(RequestFutureListener<T> listener) {
+        if (isDone) {
+            if (exception != null)
+                listener.onFailure(exception);
+            else
+                listener.onSuccess(value);
+        } else {
+            this.listeners.add(listener);
+        }
+    }
+
+    /**
+     * Convert from a request future of one type to another type
+     * @param adapter The adapter which does the conversion
+     * @param <S> The type of the future adapted to
+     * @return The new future
+     */
+    public <S> RequestFuture<S> compose(final RequestFutureAdapter<T, S> adapter) {
+        final RequestFuture<S> adapted = new RequestFuture<S>();
+        addListener(new RequestFutureListener<T>() {
+            @Override
+            public void onSuccess(T value) {
+                adapter.onSuccess(value, adapted);
+            }
+
+            @Override
+            public void onFailure(RuntimeException e) {
+                adapter.onFailure(e, adapted);
+            }
+        });
+        return adapted;
+    }
+
+    public static <T> RequestFuture<T> failure(RuntimeException e) {
+        RequestFuture<T> future = new RequestFuture<T>();
+        future.raise(e);
+        return future;
+    }
+
+    public static RequestFuture<Void> voidSuccess() {
+        RequestFuture<Void> future = new RequestFuture<Void>();
+        future.complete(null);
+        return future;
+    }
+
+    public static <T> RequestFuture<T> coordinatorNotAvailable() {
+        return failure(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception());
+    }
+
+    public static <T> RequestFuture<T> leaderNotAvailable() {
+        return failure(Errors.LEADER_NOT_AVAILABLE.exception());
+    }
+
+    public static <T> RequestFuture<T> noBrokersAvailable() {
+        return failure(new NoAvailableBrokersException());
+    }
+
+    public static <T> RequestFuture<T> staleMetadata() {
+        return failure(new StaleMetadataException());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureAdapter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureAdapter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureAdapter.java
new file mode 100644
index 0000000..b34c2da
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureAdapter.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.kafka_backport.clients.consumer.internals;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Adapt from a request future of one type to another.
+ *
+ * @param <F> Type to adapt from
+ * @param <T> Type to adapt to
+ */
+public abstract class RequestFutureAdapter<F, T> {
+
+    public abstract void onSuccess(F value, RequestFuture<T> future);
+
+    public void onFailure(RuntimeException e, RequestFuture<T> future) {
+        future.raise(e);
+    }
+}


[06/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Metrics.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Metrics.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Metrics.java
deleted file mode 100644
index 709a868..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Metrics.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.kafka.copied.common.metrics;
-
-import org.apache.kafka.copied.common.MetricName;
-import org.apache.kafka.copied.common.utils.CopyOnWriteMap;
-import org.apache.kafka.copied.common.utils.SystemTime;
-import org.apache.kafka.copied.common.utils.Time;
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.io.Closeable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * A registry of sensors and metrics.
- * <p>
- * A metric is a named, numerical measurement. A sensor is a handle to record numerical measurements as they occur. Each
- * Sensor has zero or more associated metrics. For example a Sensor might represent message sizes and we might associate
- * with this sensor a metric for the average, maximum, or other statistics computed off the sequence of message sizes
- * that are recorded by the sensor.
- * <p>
- * Usage looks something like this:
- * 
- * <pre>
- * // set up metrics:
- * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
- * Sensor sensor = metrics.sensor(&quot;message-sizes&quot;);
- * MetricName metricName = new MetricName(&quot;message-size-avg&quot;, &quot;producer-metrics&quot;);
- * sensor.add(metricName, new Avg());
- * metricName = new MetricName(&quot;message-size-max&quot;, &quot;producer-metrics&quot;);
- * sensor.add(metricName, new Max());
- * 
- * // as messages are sent we record the sizes
- * sensor.record(messageSize);
- * </pre>
- */
-public class Metrics implements Closeable {
-
-    private final MetricConfig config;
-    private final ConcurrentMap<MetricName, KafkaMetric> metrics;
-    private final ConcurrentMap<String, Sensor> sensors;
-    private final List<MetricsReporter> reporters;
-    private final Time time;
-
-    /**
-     * Create a metrics repository with no metric reporters and default configuration.
-     */
-    public Metrics() {
-        this(new MetricConfig());
-    }
-
-    /**
-     * Create a metrics repository with no metric reporters and default configuration.
-     */
-    public Metrics(Time time) {
-        this(new MetricConfig(), new ArrayList<MetricsReporter>(0), time);
-    }
-
-    /**
-     * Create a metrics repository with no reporters and the given default config. This config will be used for any
-     * metric that doesn't override its own config.
-     * @param defaultConfig The default config to use for all metrics that don't override their config
-     */
-    public Metrics(MetricConfig defaultConfig) {
-        this(defaultConfig, new ArrayList<MetricsReporter>(0), new SystemTime());
-    }
-
-    /**
-     * Create a metrics repository with a default config and the given metric reporters
-     * @param defaultConfig The default config
-     * @param reporters The metrics reporters
-     * @param time The time instance to use with the metrics
-     */
-    public Metrics(MetricConfig defaultConfig, List<MetricsReporter> reporters, Time time) {
-        this.config = defaultConfig;
-        this.sensors = new CopyOnWriteMap<String, Sensor>();
-        this.metrics = new CopyOnWriteMap<MetricName, KafkaMetric>();
-        this.reporters = Utils.notNull(reporters);
-        this.time = time;
-        for (MetricsReporter reporter : reporters)
-            reporter.init(new ArrayList<KafkaMetric>());
-    }
-
-    /**
-     * Get the sensor with the given name if it exists
-     * @param name The name of the sensor
-     * @return Return the sensor or null if no such sensor exists
-     */
-    public Sensor getSensor(String name) {
-        return this.sensors.get(Utils.notNull(name));
-    }
-
-    /**
-     * Get or create a sensor with the given unique name and no parent sensors.
-     * @param name The sensor name
-     * @return The sensor
-     */
-    public Sensor sensor(String name) {
-        return sensor(name, null, (Sensor[]) null);
-    }
-
-    /**
-     * Get or create a sensor with the given unique name and zero or more parent sensors. All parent sensors will
-     * receive every value recorded with this sensor.
-     * @param name The name of the sensor
-     * @param parents The parent sensors
-     * @return The sensor that is created
-     */
-    public Sensor sensor(String name, Sensor... parents) {
-        return sensor(name, null, parents);
-    }
-
-    /**
-     * Get or create a sensor with the given unique name and zero or more parent sensors. All parent sensors will
-     * receive every value recorded with this sensor.
-     * @param name The name of the sensor
-     * @param config A default configuration to use for this sensor for metrics that don't have their own config
-     * @param parents The parent sensors
-     * @return The sensor that is created
-     */
-    public synchronized Sensor sensor(String name, MetricConfig config, Sensor... parents) {
-        Sensor s = getSensor(name);
-        if (s == null) {
-            s = new Sensor(this, name, parents, config == null ? this.config : config, time);
-            this.sensors.put(name, s);
-        }
-        return s;
-    }
-
-    /**
-     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
-     * This is a way to expose existing values as metrics.
-     * @param metricName The name of the metric
-     * @param measurable The measurable that will be measured by this metric
-     */
-    public void addMetric(MetricName metricName, Measurable measurable) {
-        addMetric(metricName, null, measurable);
-    }
-
-    /**
-     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
-     * This is a way to expose existing values as metrics.
-     * @param metricName The name of the metric
-     * @param config The configuration to use when measuring this measurable
-     * @param measurable The measurable that will be measured by this metric
-     */
-    public synchronized void addMetric(MetricName metricName, MetricConfig config, Measurable measurable) {
-        KafkaMetric m = new KafkaMetric(new Object(),
-                                        Utils.notNull(metricName),
-                                        Utils.notNull(measurable),
-                                        config == null ? this.config : config,
-                                        time);
-        registerMetric(m);
-    }
-
-    /**
-     * Add a MetricReporter
-     */
-    public synchronized void addReporter(MetricsReporter reporter) {
-        Utils.notNull(reporter).init(new ArrayList<KafkaMetric>(metrics.values()));
-        this.reporters.add(reporter);
-    }
-
-    synchronized void registerMetric(KafkaMetric metric) {
-        MetricName metricName = metric.metricName();
-        if (this.metrics.containsKey(metricName))
-            throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one.");
-        this.metrics.put(metricName, metric);
-        for (MetricsReporter reporter : reporters)
-            reporter.metricChange(metric);
-    }
-
-    /**
-     * Get all the metrics currently maintained indexed by metricName
-     */
-    public Map<MetricName, KafkaMetric> metrics() {
-        return this.metrics;
-    }
-
-    /**
-     * Close this metrics repository.
-     */
-    @Override
-    public void close() {
-        for (MetricsReporter reporter : this.reporters)
-            reporter.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MetricsReporter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MetricsReporter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MetricsReporter.java
deleted file mode 100644
index 4f5b00d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MetricsReporter.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.kafka.copied.common.metrics;
-
-import org.apache.kafka.copied.common.Configurable;
-
-import java.util.List;
-
-/**
- * A plugin interface to allow things to listen as new metrics are created so they can be reported.
- */
-public interface MetricsReporter extends Configurable {
-
-    /**
-     * This is called when the reporter is first registered to initially register all existing metrics
-     * @param metrics All currently existing metrics
-     */
-    public void init(List<KafkaMetric> metrics);
-
-    /**
-     * This is called whenever a metric is updated or added
-     * @param metric
-     */
-    public void metricChange(KafkaMetric metric);
-
-    /**
-     * Called when the metrics repository is closed.
-     */
-    public void close();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Quota.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Quota.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Quota.java
deleted file mode 100644
index f9893a1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Quota.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.kafka.copied.common.metrics;
-
-/**
- * An upper or lower bound for metrics
- */
-public final class Quota {
-
-    private final boolean upper;
-    private final double bound;
-
-    public Quota(double bound, boolean upper) {
-        this.bound = bound;
-        this.upper = upper;
-    }
-
-    public static Quota lessThan(double upperBound) {
-        return new Quota(upperBound, true);
-    }
-
-    public static Quota moreThan(double lowerBound) {
-        return new Quota(lowerBound, false);
-    }
-
-    public boolean isUpperBound() {
-        return this.upper;
-    }
-
-    public double bound() {
-        return this.bound;
-    }
-
-    public boolean acceptable(double value) {
-        return (upper && value <= bound) || (!upper && value >= bound);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/QuotaViolationException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/QuotaViolationException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/QuotaViolationException.java
deleted file mode 100644
index add99b9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/QuotaViolationException.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.kafka.copied.common.metrics;
-
-import org.apache.kafka.copied.common.KafkaException;
-
-/**
- * Thrown when a sensor records a value that causes a metric to go outside the bounds configured as its quota
- */
-public class QuotaViolationException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public QuotaViolationException(String m) {
-        super(m);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Sensor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Sensor.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Sensor.java
deleted file mode 100644
index e4df999..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Sensor.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.metrics;
-
-import org.apache.kafka.copied.common.MetricName;
-import org.apache.kafka.copied.common.utils.Time;
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-/**
- * A sensor applies a continuous sequence of numerical values to a set of associated metrics. For example a sensor on
- * message size would record a sequence of message sizes using the {@link #record(double)} api and would maintain a set
- * of metrics about request sizes such as the average or max.
- */
-public final class Sensor {
-
-    private final Metrics registry;
-    private final String name;
-    private final Sensor[] parents;
-    private final List<Stat> stats;
-    private final List<KafkaMetric> metrics;
-    private final MetricConfig config;
-    private final Time time;
-
-    Sensor(Metrics registry, String name, Sensor[] parents, MetricConfig config, Time time) {
-        super();
-        this.registry = registry;
-        this.name = Utils.notNull(name);
-        this.parents = parents == null ? new Sensor[0] : parents;
-        this.metrics = new ArrayList<KafkaMetric>();
-        this.stats = new ArrayList<Stat>();
-        this.config = config;
-        this.time = time;
-        checkForest(new HashSet<Sensor>());
-    }
-
-    /* Validate that this sensor doesn't end up referencing itself */
-    private void checkForest(Set<Sensor> sensors) {
-        if (!sensors.add(this))
-            throw new IllegalArgumentException("Circular dependency in sensors: " + name() + " is its own parent.");
-        for (int i = 0; i < parents.length; i++)
-            parents[i].checkForest(sensors);
-    }
-
-    /**
-     * The name this sensor is registered with. This name will be unique among all registered sensors.
-     */
-    public String name() {
-        return this.name;
-    }
-
-    /**
-     * Record an occurrence, this is just short-hand for {@link #record(double) record(1.0)}
-     */
-    public void record() {
-        record(1.0);
-    }
-
-    /**
-     * Record a value with this sensor
-     * @param value The value to record
-     * @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum
-     *         bound
-     */
-    public void record(double value) {
-        record(value, time.milliseconds());
-    }
-
-    /**
-     * Record a value at a known time. This method is slightly faster than {@link #record(double)} since it will reuse
-     * the time stamp.
-     * @param value The value we are recording
-     * @param timeMs The current POSIX time in milliseconds
-     * @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum
-     *         bound
-     */
-    public void record(double value, long timeMs) {
-        synchronized (this) {
-            // increment all the stats
-            for (int i = 0; i < this.stats.size(); i++)
-                this.stats.get(i).record(config, value, timeMs);
-            checkQuotas(timeMs);
-        }
-        for (int i = 0; i < parents.length; i++)
-            parents[i].record(value, timeMs);
-    }
-
-    /**
-     * Check if we have violated our quota for any metric that has a configured quota
-     * @param timeMs
-     */
-    private void checkQuotas(long timeMs) {
-        for (int i = 0; i < this.metrics.size(); i++) {
-            KafkaMetric metric = this.metrics.get(i);
-            MetricConfig config = metric.config();
-            if (config != null) {
-                Quota quota = config.quota();
-                if (quota != null) {
-                    if (!quota.acceptable(metric.value(timeMs)))
-                        throw new QuotaViolationException(metric.metricName() + " is in violation of its quota of " + quota.bound());
-                }
-            }
-        }
-    }
-
-    /**
-     * Register a compound statistic with this sensor with no config override
-     */
-    public void add(CompoundStat stat) {
-        add(stat, null);
-    }
-
-    /**
-     * Register a compound statistic with this sensor which yields multiple measurable quantities (like a histogram)
-     * @param stat The stat to register
-     * @param config The configuration for this stat. If null then the stat will use the default configuration for this
-     *        sensor.
-     */
-    public synchronized void add(CompoundStat stat, MetricConfig config) {
-        this.stats.add(Utils.notNull(stat));
-        for (CompoundStat.NamedMeasurable m : stat.stats()) {
-            KafkaMetric metric = new KafkaMetric(this, m.name(), m.stat(), config == null ? this.config : config, time);
-            this.registry.registerMetric(metric);
-            this.metrics.add(metric);
-        }
-    }
-
-    /**
-     * Register a metric with this sensor
-     * @param metricName The name of the metric
-     * @param stat The statistic to keep
-     */
-    public void add(MetricName metricName, MeasurableStat stat) {
-        add(metricName, stat, null);
-    }
-
-    /**
-     * Register a metric with this sensor
-     * @param metricName The name of the metric
-     * @param stat The statistic to keep
-     * @param config A special configuration for this metric. If null use the sensor default configuration.
-     */
-    public synchronized void add(MetricName metricName, MeasurableStat stat, MetricConfig config) {
-        KafkaMetric metric = new KafkaMetric(new Object(),
-                                             Utils.notNull(metricName),
-                                             Utils.notNull(stat),
-                                             config == null ? this.config : config,
-                                             time);
-        this.registry.registerMetric(metric);
-        this.metrics.add(metric);
-        this.stats.add(stat);
-    }
-
-    synchronized List<KafkaMetric> metrics() {
-        return Collections.unmodifiableList(this.metrics);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Stat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Stat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Stat.java
deleted file mode 100644
index 67ee79b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Stat.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.kafka.copied.common.metrics;
-
-/**
- * A Stat is a quanity such as average, max, etc that is computed off the stream of updates to a sensor
- */
-public interface Stat {
-
-    /**
-     * Record the given value
-     * @param config The configuration to use for this metric
-     * @param value The value to record
-     * @param timeMs The POSIX time in milliseconds this value occurred
-     */
-    public void record(MetricConfig config, double value, long timeMs);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Avg.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Avg.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Avg.java
deleted file mode 100644
index b76f6fe..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Avg.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.kafka.copied.common.metrics.stats;
-
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-
-import java.util.List;
-
-/**
- * A {@link SampledStat} that maintains a simple average over its samples.
- */
-public class Avg extends SampledStat {
-
-    public Avg() {
-        super(0.0);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long now) {
-        sample.value += value;
-    }
-
-    @Override
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        double total = 0.0;
-        long count = 0;
-        for (int i = 0; i < samples.size(); i++) {
-            Sample s = samples.get(i);
-            total += s.value;
-            count += s.eventCount;
-        }
-        return total / count;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Count.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Count.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Count.java
deleted file mode 100644
index 4fd1c57..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Count.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.kafka.copied.common.metrics.stats;
-
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-
-import java.util.List;
-
-/**
- * A {@link SampledStat} that maintains a simple count of what it has seen.
- */
-public class Count extends SampledStat {
-
-    public Count() {
-        super(0);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long now) {
-        sample.value += 1.0;
-    }
-
-    @Override
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        double total = 0.0;
-        for (int i = 0; i < samples.size(); i++)
-            total += samples.get(i).value;
-        return total;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Histogram.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Histogram.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Histogram.java
deleted file mode 100644
index 13255c6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Histogram.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.metrics.stats;
-
-public class Histogram {
-
-    private final BinScheme binScheme;
-    private final float[] hist;
-    private double count;
-
-    public Histogram(BinScheme binScheme) {
-        this.hist = new float[binScheme.bins()];
-        this.count = 0.0f;
-        this.binScheme = binScheme;
-    }
-
-    public void record(double value) {
-        this.hist[binScheme.toBin(value)] += 1.0f;
-        this.count += 1.0f;
-    }
-
-    public double value(double quantile) {
-        if (count == 0.0d)
-            return Double.NaN;
-        float sum = 0.0f;
-        float quant = (float) quantile;
-        for (int i = 0; i < this.hist.length - 1; i++) {
-            sum += this.hist[i];
-            if (sum / count > quant)
-                return binScheme.fromBin(i);
-        }
-        return Float.POSITIVE_INFINITY;
-    }
-
-    public float[] counts() {
-        return this.hist;
-    }
-
-    public void clear() {
-        for (int i = 0; i < this.hist.length; i++)
-            this.hist[i] = 0.0f;
-        this.count = 0;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder b = new StringBuilder("{");
-        for (int i = 0; i < this.hist.length - 1; i++) {
-            b.append(String.format("%.10f", binScheme.fromBin(i)));
-            b.append(':');
-            b.append(String.format("%.0f", this.hist[i]));
-            b.append(',');
-        }
-        b.append(Float.POSITIVE_INFINITY);
-        b.append(':');
-        b.append(this.hist[this.hist.length - 1]);
-        b.append('}');
-        return b.toString();
-    }
-
-    public interface BinScheme {
-        public int bins();
-
-        public int toBin(double value);
-
-        public double fromBin(int bin);
-    }
-
-    public static class ConstantBinScheme implements BinScheme {
-        private final double min;
-        private final double max;
-        private final int bins;
-        private final double bucketWidth;
-
-        public ConstantBinScheme(int bins, double min, double max) {
-            if (bins < 2)
-                throw new IllegalArgumentException("Must have at least 2 bins.");
-            this.min = min;
-            this.max = max;
-            this.bins = bins;
-            this.bucketWidth = (max - min) / (bins - 2);
-        }
-
-        public int bins() {
-            return this.bins;
-        }
-
-        public double fromBin(int b) {
-            if (b == 0)
-                return Double.NEGATIVE_INFINITY;
-            else if (b == bins - 1)
-                return Double.POSITIVE_INFINITY;
-            else
-                return min + (b - 1) * bucketWidth;
-        }
-
-        public int toBin(double x) {
-            if (x < min)
-                return 0;
-            else if (x > max)
-                return bins - 1;
-            else
-                return (int) ((x - min) / bucketWidth) + 1;
-        }
-    }
-
-    public static class LinearBinScheme implements BinScheme {
-        private final int bins;
-        private final double max;
-        private final double scale;
-
-        public LinearBinScheme(int numBins, double max) {
-            this.bins = numBins;
-            this.max = max;
-            this.scale = max / (numBins * (numBins - 1) / 2);
-        }
-
-        public int bins() {
-            return this.bins;
-        }
-
-        public double fromBin(int b) {
-            if (b == this.bins - 1) {
-                return Float.POSITIVE_INFINITY;
-            } else {
-                double unscaled = (b * (b + 1.0)) / 2.0;
-                return unscaled * this.scale;
-            }
-        }
-
-        public int toBin(double x) {
-            if (x < 0.0d) {
-                throw new IllegalArgumentException("Values less than 0.0 not accepted.");
-            } else if (x > this.max) {
-                return this.bins - 1;
-            } else {
-                double scaled = x / this.scale;
-                return (int) (-0.5 + Math.sqrt(2.0 * scaled + 0.25));
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Max.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Max.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Max.java
deleted file mode 100644
index 8b1d8d0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Max.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.kafka.copied.common.metrics.stats;
-
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-
-import java.util.List;
-
-/**
- * A {@link SampledStat} that gives the max over its samples.
- */
-public final class Max extends SampledStat {
-
-    public Max() {
-        super(Double.NEGATIVE_INFINITY);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long now) {
-        sample.value = Math.max(sample.value, value);
-    }
-
-    @Override
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        double max = Double.NEGATIVE_INFINITY;
-        for (int i = 0; i < samples.size(); i++)
-            max = Math.max(max, samples.get(i).value);
-        return max;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Min.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Min.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Min.java
deleted file mode 100644
index b4af5f8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Min.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.kafka.copied.common.metrics.stats;
-
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-
-import java.util.List;
-
-/**
- * A {@link SampledStat} that gives the min over its samples.
- */
-public class Min extends SampledStat {
-
-    public Min() {
-        super(Double.MIN_VALUE);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long now) {
-        sample.value = Math.min(sample.value, value);
-    }
-
-    @Override
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        double max = Double.MAX_VALUE;
-        for (int i = 0; i < samples.size(); i++)
-            max = Math.min(max, samples.get(i).value);
-        return max;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Percentile.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Percentile.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Percentile.java
deleted file mode 100644
index dac44eb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Percentile.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.kafka.copied.common.metrics.stats;
-
-import org.apache.kafka.copied.common.MetricName;
-
-public class Percentile {
-
-    private final MetricName name;
-    private final double percentile;
-
-    public Percentile(MetricName name, double percentile) {
-        super();
-        this.name = name;
-        this.percentile = percentile;
-    }
-
-    public MetricName name() {
-        return this.name;
-    }
-
-    public double percentile() {
-        return this.percentile;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Percentiles.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Percentiles.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Percentiles.java
deleted file mode 100644
index ed94418..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Percentiles.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.metrics.stats;
-
-import org.apache.kafka.copied.common.metrics.CompoundStat;
-import org.apache.kafka.copied.common.metrics.Measurable;
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-import org.apache.kafka.copied.common.metrics.stats.Histogram.BinScheme;
-import org.apache.kafka.copied.common.metrics.stats.Histogram.ConstantBinScheme;
-import org.apache.kafka.copied.common.metrics.stats.Histogram.LinearBinScheme;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * A compound stat that reports one or more percentiles
- */
-public class Percentiles extends SampledStat implements CompoundStat {
-
-    public static enum BucketSizing {
-        CONSTANT, LINEAR
-    }
-
-    private final int buckets;
-    private final Percentile[] percentiles;
-    private final BinScheme binScheme;
-
-    public Percentiles(int sizeInBytes, double max, BucketSizing bucketing, Percentile... percentiles) {
-        this(sizeInBytes, 0.0, max, bucketing, percentiles);
-    }
-
-    public Percentiles(int sizeInBytes, double min, double max, BucketSizing bucketing, Percentile... percentiles) {
-        super(0.0);
-        this.percentiles = percentiles;
-        this.buckets = sizeInBytes / 4;
-        if (bucketing == BucketSizing.CONSTANT) {
-            this.binScheme = new ConstantBinScheme(buckets, min, max);
-        } else if (bucketing == BucketSizing.LINEAR) {
-            if (min != 0.0d)
-                throw new IllegalArgumentException("Linear bucket sizing requires min to be 0.0.");
-            this.binScheme = new LinearBinScheme(buckets, max);
-        } else {
-            throw new IllegalArgumentException("Unknown bucket type: " + bucketing);
-        }
-    }
-
-    @Override
-    public List<NamedMeasurable> stats() {
-        List<NamedMeasurable> ms = new ArrayList<NamedMeasurable>(this.percentiles.length);
-        for (Percentile percentile : this.percentiles) {
-            final double pct = percentile.percentile();
-            ms.add(new NamedMeasurable(percentile.name(), new Measurable() {
-                public double measure(MetricConfig config, long now) {
-                    return value(config, now, pct / 100.0);
-                }
-            }));
-        }
-        return ms;
-    }
-
-    public double value(MetricConfig config, long now, double quantile) {
-        purgeObsoleteSamples(config, now);
-        float count = 0.0f;
-        for (Sample sample : this.samples)
-            count += sample.eventCount;
-        if (count == 0.0f)
-            return Double.NaN;
-        float sum = 0.0f;
-        float quant = (float) quantile;
-        for (int b = 0; b < buckets; b++) {
-            for (int s = 0; s < this.samples.size(); s++) {
-                HistogramSample sample = (HistogramSample) this.samples.get(s);
-                float[] hist = sample.histogram.counts();
-                sum += hist[b];
-                if (sum / count > quant)
-                    return binScheme.fromBin(b);
-            }
-        }
-        return Double.POSITIVE_INFINITY;
-    }
-
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        return value(config, now, 0.5);
-    }
-
-    @Override
-    protected HistogramSample newSample(long timeMs) {
-        return new HistogramSample(this.binScheme, timeMs);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long timeMs) {
-        HistogramSample hist = (HistogramSample) sample;
-        hist.histogram.record(value);
-    }
-
-    private static class HistogramSample extends SampledStat.Sample {
-        private final Histogram histogram;
-
-        private HistogramSample(BinScheme scheme, long now) {
-            super(0.0, now);
-            this.histogram = new Histogram(scheme);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Rate.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Rate.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Rate.java
deleted file mode 100644
index 2eb6d64..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Rate.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.kafka.copied.common.metrics.stats;
-
-import org.apache.kafka.copied.common.metrics.MeasurableStat;
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-/**
- * The rate of the given quantity. By default this is the total observed over a set of samples from a sampled statistic
- * divided by the elapsed time over the sample windows. Alternative {@link SampledStat} implementations can be provided,
- * however, to record the rate of occurrences (e.g. the count of values measured over the time interval) or other such
- * values.
- */
-public class Rate implements MeasurableStat {
-
-    private final TimeUnit unit;
-    private final SampledStat stat;
-
-    public Rate() {
-        this(TimeUnit.SECONDS);
-    }
-
-    public Rate(TimeUnit unit) {
-        this(unit, new SampledTotal());
-    }
-
-    public Rate(SampledStat stat) {
-        this(TimeUnit.SECONDS, stat);
-    }
-
-    public Rate(TimeUnit unit, SampledStat stat) {
-        this.stat = stat;
-        this.unit = unit;
-    }
-
-    public String unitName() {
-        return unit.name().substring(0, unit.name().length() - 2).toLowerCase();
-    }
-
-    @Override
-    public void record(MetricConfig config, double value, long timeMs) {
-        this.stat.record(config, value, timeMs);
-    }
-
-    @Override
-    public double measure(MetricConfig config, long now) {
-        double value = stat.measure(config, now);
-        double elapsed = convert(now - stat.oldest(now).lastWindowMs);
-        return value / elapsed;
-    }
-
-    private double convert(long time) {
-        switch (unit) {
-            case NANOSECONDS:
-                return time * 1000.0 * 1000.0;
-            case MICROSECONDS:
-                return time * 1000.0;
-            case MILLISECONDS:
-                return time;
-            case SECONDS:
-                return time / 1000.0;
-            case MINUTES:
-                return time / (60.0 * 1000.0);
-            case HOURS:
-                return time / (60.0 * 60.0 * 1000.0);
-            case DAYS:
-                return time / (24.0 * 60.0 * 60.0 * 1000.0);
-            default:
-                throw new IllegalStateException("Unknown unit: " + unit);
-        }
-    }
-
-    public static class SampledTotal extends SampledStat {
-
-        public SampledTotal() {
-            super(0.0d);
-        }
-
-        @Override
-        protected void update(Sample sample, MetricConfig config, double value, long timeMs) {
-            sample.value += value;
-        }
-
-        @Override
-        public double combine(List<Sample> samples, MetricConfig config, long now) {
-            double total = 0.0;
-            for (int i = 0; i < samples.size(); i++)
-                total += samples.get(i).value;
-            return total;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/SampledStat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/SampledStat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/SampledStat.java
deleted file mode 100644
index 6d53a89..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/SampledStat.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.metrics.stats;
-
-import org.apache.kafka.copied.common.metrics.MeasurableStat;
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * A SampledStat records a single scalar value measured over one or more samples. Each sample is recorded over a
- * configurable window. The window can be defined by number of events or ellapsed time (or both, if both are given the
- * window is complete when <i>either</i> the event count or ellapsed time criterion is met).
- * <p>
- * All the samples are combined to produce the measurement. When a window is complete the oldest sample is cleared and
- * recycled to begin recording the next sample.
- * 
- * Subclasses of this class define different statistics measured using this basic pattern.
- */
-public abstract class SampledStat implements MeasurableStat {
-
-    private double initialValue;
-    private int current = 0;
-    protected List<Sample> samples;
-
-    public SampledStat(double initialValue) {
-        this.initialValue = initialValue;
-        this.samples = new ArrayList<Sample>(2);
-    }
-
-    @Override
-    public void record(MetricConfig config, double value, long timeMs) {
-        Sample sample = current(timeMs);
-        if (sample.isComplete(timeMs, config))
-            sample = advance(config, timeMs);
-        update(sample, config, value, timeMs);
-        sample.eventCount += 1;
-    }
-
-    private Sample advance(MetricConfig config, long timeMs) {
-        this.current = (this.current + 1) % config.samples();
-        if (this.current >= samples.size()) {
-            Sample sample = newSample(timeMs);
-            this.samples.add(sample);
-            return sample;
-        } else {
-            Sample sample = current(timeMs);
-            sample.reset(timeMs);
-            return sample;
-        }
-    }
-
-    protected Sample newSample(long timeMs) {
-        return new Sample(this.initialValue, timeMs);
-    }
-
-    @Override
-    public double measure(MetricConfig config, long now) {
-        purgeObsoleteSamples(config, now);
-        return combine(this.samples, config, now);
-    }
-
-    public Sample current(long timeMs) {
-        if (samples.size() == 0)
-            this.samples.add(newSample(timeMs));
-        return this.samples.get(this.current);
-    }
-
-    public Sample oldest(long now) {
-        if (samples.size() == 0)
-            this.samples.add(newSample(now));
-        Sample oldest = this.samples.get(0);
-        for (int i = 1; i < this.samples.size(); i++) {
-            Sample curr = this.samples.get(i);
-            if (curr.lastWindowMs < oldest.lastWindowMs)
-                oldest = curr;
-        }
-        return oldest;
-    }
-
-    protected abstract void update(Sample sample, MetricConfig config, double value, long timeMs);
-
-    public abstract double combine(List<Sample> samples, MetricConfig config, long now);
-
-    /* Timeout any windows that have expired in the absence of any events */
-    protected void purgeObsoleteSamples(MetricConfig config, long now) {
-        long expireAge = config.samples() * config.timeWindowMs();
-        for (int i = 0; i < samples.size(); i++) {
-            Sample sample = this.samples.get(i);
-            if (now - sample.lastWindowMs >= expireAge)
-                sample.reset(now);
-        }
-    }
-
-    protected static class Sample {
-        public double initialValue;
-        public long eventCount;
-        public long lastWindowMs;
-        public double value;
-
-        public Sample(double initialValue, long now) {
-            this.initialValue = initialValue;
-            this.eventCount = 0;
-            this.lastWindowMs = now;
-            this.value = initialValue;
-        }
-
-        public void reset(long now) {
-            this.eventCount = 0;
-            this.lastWindowMs = now;
-            this.value = initialValue;
-        }
-
-        public boolean isComplete(long timeMs, MetricConfig config) {
-            return timeMs - lastWindowMs >= config.timeWindowMs() || eventCount >= config.eventWindow();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Total.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Total.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Total.java
deleted file mode 100644
index 98909b1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/stats/Total.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.kafka.copied.common.metrics.stats;
-
-import org.apache.kafka.copied.common.metrics.MeasurableStat;
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-
-/**
- * An un-windowed cumulative total maintained over all time.
- */
-public class Total implements MeasurableStat {
-
-    private double total;
-
-    public Total() {
-        this.total = 0.0;
-    }
-
-    public Total(double value) {
-        this.total = value;
-    }
-
-    @Override
-    public void record(MetricConfig config, double value, long now) {
-        this.total += value;
-    }
-
-    @Override
-    public double measure(MetricConfig config, long now) {
-        return this.total;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/ByteBufferReceive.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/ByteBufferReceive.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/ByteBufferReceive.java
deleted file mode 100644
index 6ae4dcd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/ByteBufferReceive.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.kafka.copied.common.network;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ScatteringByteChannel;
-
-/**
- * A receive backed by an array of ByteBuffers
- */
-public class ByteBufferReceive implements Receive {
-
-    private final String source;
-    private final ByteBuffer[] buffers;
-    private int remaining;
-
-    public ByteBufferReceive(String source, ByteBuffer... buffers) {
-        super();
-        this.source = source;
-        this.buffers = buffers;
-        for (int i = 0; i < buffers.length; i++)
-            remaining += buffers[i].remaining();
-    }
-
-    @Override
-    public String source() {
-        return source;
-    }
-
-    @Override
-    public boolean complete() {
-        return remaining > 0;
-    }
-
-    @Override
-    public long readFrom(ScatteringByteChannel channel) throws IOException {
-        long read = channel.read(buffers);
-        remaining += read;
-        return read;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/ByteBufferSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/ByteBufferSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/ByteBufferSend.java
deleted file mode 100644
index c573db5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/ByteBufferSend.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.kafka.copied.common.network;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.GatheringByteChannel;
-
-/**
- * A send backed by an array of byte buffers
- */
-public class ByteBufferSend implements Send {
-
-    private final String destination;
-    protected final ByteBuffer[] buffers;
-    private int remaining;
-    private int size;
-
-    public ByteBufferSend(String destination, ByteBuffer... buffers) {
-        super();
-        this.destination = destination;
-        this.buffers = buffers;
-        for (int i = 0; i < buffers.length; i++)
-            remaining += buffers[i].remaining();
-        this.size = remaining;
-    }
-
-    @Override
-    public String destination() {
-        return destination;
-    }
-
-    @Override
-    public boolean completed() {
-        return remaining <= 0;
-    }
-
-    @Override
-    public long size() {
-        return this.size;
-    }
-
-    @Override
-    public long writeTo(GatheringByteChannel channel) throws IOException {
-        long written = channel.write(buffers);
-        if (written < 0)
-            throw new EOFException("Wrote negative bytes to channel. This shouldn't happen.");
-        remaining -= written;
-        return written;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/InvalidReceiveException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/InvalidReceiveException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/InvalidReceiveException.java
deleted file mode 100644
index 24dc983..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/InvalidReceiveException.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.network;
-
-import org.apache.kafka.copied.common.KafkaException;
-
-public class InvalidReceiveException extends KafkaException {
-
-    public InvalidReceiveException(String message) {
-        super(message);
-    }
-
-    public InvalidReceiveException(String message, Throwable cause) {
-        super(message, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/MultiSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/MultiSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/MultiSend.java
deleted file mode 100644
index 38541e2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/MultiSend.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.network;
-
-import org.apache.kafka.copied.common.KafkaException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.nio.channels.GatheringByteChannel;
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * A set of composite sends, sent one after another
- */
-
-public class MultiSend implements Send {
-
-    private static final Logger log = LoggerFactory.getLogger(MultiSend.class);
-    private String dest;
-    private long totalWritten = 0;
-    private List<Send> sends;
-    private Iterator<Send> sendsIterator;
-    private Send current;
-    private boolean doneSends = false;
-    private long size = 0;
-
-    public MultiSend(String dest, List<Send> sends) {
-        this.dest = dest;
-        this.sends = sends;
-        this.sendsIterator = sends.iterator();
-        nextSendOrDone();
-        for (Send send: sends)
-            this.size += send.size();
-    }
-
-    @Override
-    public long size() {
-        return size;
-    }
-
-    @Override
-    public String destination() {
-        return dest;
-    }
-
-    @Override
-    public boolean completed() {
-        if (doneSends) {
-            if (totalWritten != size)
-                log.error("mismatch in sending bytes over socket; expected: " + size + " actual: " + totalWritten);
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public long writeTo(GatheringByteChannel channel) throws IOException {
-        if (completed())
-            throw new KafkaException("This operation cannot be completed on a complete request.");
-
-        int totalWrittenPerCall = 0;
-        boolean sendComplete = false;
-        do {
-            long written = current.writeTo(channel);
-            totalWritten += written;
-            totalWrittenPerCall += written;
-            sendComplete = current.completed();
-            if (sendComplete)
-                nextSendOrDone();
-        } while (!completed() && sendComplete);
-        if (log.isTraceEnabled())
-            log.trace("Bytes written as part of multisend call : " + totalWrittenPerCall +  "Total bytes written so far : " + totalWritten + "Expected bytes to write : " + size);
-        return totalWrittenPerCall;
-    }
-
-    // update current if there's a next Send, mark sends as done if there isn't
-    private void nextSendOrDone() {
-        if (sendsIterator.hasNext())
-            current = sendsIterator.next();
-        else
-            doneSends = true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/NetworkReceive.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/NetworkReceive.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/NetworkReceive.java
deleted file mode 100644
index 6b065f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/NetworkReceive.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.kafka.copied.common.network;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.ScatteringByteChannel;
-
-/**
- * A size delimited Receive that consists of a 4 byte network-ordered size N followed by N bytes of content
- */
-public class NetworkReceive implements Receive {
-
-    public final static String UNKNOWN_SOURCE = "";
-    public final static int UNLIMITED = -1;
-
-    private final String source;
-    private final ByteBuffer size;
-    private final int maxSize;
-    private ByteBuffer buffer;
-
-
-    public NetworkReceive(String source, ByteBuffer buffer) {
-        this.source = source;
-        this.buffer = buffer;
-        this.size = null;
-        this.maxSize = UNLIMITED;
-    }
-
-    public NetworkReceive(String source) {
-        this.source = source;
-        this.size = ByteBuffer.allocate(4);
-        this.buffer = null;
-        this.maxSize = UNLIMITED;
-    }
-
-    public NetworkReceive(int maxSize, String source) {
-        this.source = source;
-        this.size = ByteBuffer.allocate(4);
-        this.buffer = null;
-        this.maxSize = maxSize;
-    }
-
-    public NetworkReceive() {
-        this(UNKNOWN_SOURCE);
-    }
-
-    @Override
-    public String source() {
-        return source;
-    }
-
-    @Override
-    public boolean complete() {
-        return !size.hasRemaining() && !buffer.hasRemaining();
-    }
-
-    public long readFrom(ScatteringByteChannel channel) throws IOException {
-        return readFromReadableChannel(channel);
-    }
-
-    // Need a method to read from ReadableByteChannel because BlockingChannel requires read with timeout
-    // See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work
-    // This can go away after we get rid of BlockingChannel
-    @Deprecated
-    public long readFromReadableChannel(ReadableByteChannel channel) throws IOException {
-        int read = 0;
-        if (size.hasRemaining()) {
-            int bytesRead = channel.read(size);
-            if (bytesRead < 0)
-                throw new EOFException();
-            read += bytesRead;
-            if (!size.hasRemaining()) {
-                size.rewind();
-                int receiveSize = size.getInt();
-                if (receiveSize < 0)
-                    throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + ")");
-                if (maxSize != UNLIMITED && receiveSize > maxSize)
-                    throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + " larger than " + maxSize + ")");
-                this.buffer = ByteBuffer.allocate(receiveSize);
-            }
-        }
-        if (buffer != null) {
-            int bytesRead = channel.read(buffer);
-            if (bytesRead < 0)
-                throw new EOFException();
-            read += bytesRead;
-        }
-
-        return read;
-    }
-
-    public ByteBuffer payload() {
-        return this.buffer;
-    }
-
-    // Used only by BlockingChannel, so we may be able to get rid of this when/if we get rid of BlockingChannel
-    @Deprecated
-    public long readCompletely(ReadableByteChannel channel) throws IOException {
-        int totalRead = 0;
-        while (!complete()) {
-            totalRead += readFromReadableChannel(channel);
-        }
-        return totalRead;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/NetworkSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/NetworkSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/NetworkSend.java
deleted file mode 100644
index 1c8438c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/NetworkSend.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.network;
-
-import java.nio.ByteBuffer;
-
-/**
- * A size delimited Send that consists of a 4 byte network-ordered size N followed by N bytes of content
- */
-public class NetworkSend extends ByteBufferSend {
-
-    public NetworkSend(String destination, ByteBuffer... buffers) {
-        super(destination, sizeDelimit(buffers));
-    }
-
-    private static ByteBuffer[] sizeDelimit(ByteBuffer[] buffers) {
-        int size = 0;
-        for (int i = 0; i < buffers.length; i++)
-            size += buffers[i].remaining();
-        ByteBuffer[] delimited = new ByteBuffer[buffers.length + 1];
-        delimited[0] = ByteBuffer.allocate(4);
-        delimited[0].putInt(size);
-        delimited[0].rewind();
-        System.arraycopy(buffers, 0, delimited, 1, buffers.length);
-        return delimited;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Receive.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Receive.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Receive.java
deleted file mode 100644
index cb66b57..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Receive.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.kafka.copied.common.network;
-
-import java.io.IOException;
-import java.nio.channels.ScatteringByteChannel;
-
-/**
- * This interface models the in-progress reading of data from a channel to a source identified by an integer id
- */
-public interface Receive {
-
-    /**
-     * The numeric id of the source from which we are receiving data.
-     */
-    public String source();
-
-    /**
-     * Are we done receiving data?
-     */
-    public boolean complete();
-
-    /**
-     * Read bytes into this receive from the given channel
-     * @param channel The channel to read from
-     * @return The number of bytes read
-     * @throws IOException If the reading fails
-     */
-    public long readFrom(ScatteringByteChannel channel) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Selectable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Selectable.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Selectable.java
deleted file mode 100644
index b5bc46e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Selectable.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.kafka.copied.common.network;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.List;
-
-/**
- * An interface for asynchronous, multi-channel network I/O
- */
-public interface Selectable {
-
-    /**
-     * Begin establishing a socket connection to the given address identified by the given address
-     * @param id The id for this connection
-     * @param address The address to connect to
-     * @param sendBufferSize The send buffer for the socket
-     * @param receiveBufferSize The receive buffer for the socket
-     * @throws IOException If we cannot begin connecting
-     */
-    public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException;
-
-    /**
-     * Begin disconnecting the connection identified by the given id
-     */
-    public void disconnect(String id);
-
-    /**
-     * Wakeup this selector if it is blocked on I/O
-     */
-    public void wakeup();
-
-    /**
-     * Close this selector
-     */
-    public void close();
-
-    /**
-     * Queue the given request for sending in the subsequent {@poll(long)} calls
-     * @param send The request to send
-     */
-    public void send(Send send);
-
-    /**
-     * Do I/O. Reads, writes, connection establishment, etc.
-     * @param timeout The amount of time to block if there is nothing to do
-     * @throws IOException
-     */
-    public void poll(long timeout) throws IOException;
-
-    /**
-     * The list of sends that completed on the last {@link #poll(long, List) poll()} call.
-     */
-    public List<Send> completedSends();
-
-    /**
-     * The list of receives that completed on the last {@link #poll(long, List) poll()} call.
-     */
-    public List<NetworkReceive> completedReceives();
-
-    /**
-     * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()}
-     * call.
-     */
-    public List<String> disconnected();
-
-    /**
-     * The list of connections that completed their connection on the last {@link #poll(long, List) poll()}
-     * call.
-     */
-    public List<String> connected();
-
-    /**
-     * Disable reads from the given connection
-     * @param id The id for the connection
-     */
-    public void mute(String id);
-
-    /**
-     * Re-enable reads from the given connection
-     * @param id The id for the connection
-     */
-    public void unmute(String id);
-
-    /**
-     * Disable reads from all connections
-     */
-    public void muteAll();
-
-    /**
-     * Re-enable reads from all connections
-     */
-    public void unmuteAll();
-
-}
\ No newline at end of file


[08/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFuture.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFuture.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFuture.java
deleted file mode 100644
index a2a716a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFuture.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.kafka.copied.clients.consumer.internals;
-
-import org.apache.kafka.copied.common.errors.RetriableException;
-import org.apache.kafka.copied.common.protocol.Errors;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Result of an asynchronous request from {@link ConsumerNetworkClient}. Use {@link ConsumerNetworkClient#poll(long)}
- * (and variants) to finish a request future. Use {@link #isDone()} to check if the future is complete, and
- * {@link #succeeded()} to check if the request completed successfully. Typical usage might look like this:
- *
- * <pre>
- *     RequestFuture<ClientResponse> future = client.send(api, request);
- *     client.poll(future);
- *
- *     if (future.succeeded()) {
- *         ClientResponse response = future.value();
- *         // Handle response
- *     } else {
- *         throw future.exception();
- *     }
- * </pre>
- *
- * @param <T> Return type of the result (Can be Void if there is no response)
- */
-public class RequestFuture<T> {
-
-    private boolean isDone = false;
-    private T value;
-    private RuntimeException exception;
-    private List<RequestFutureListener<T>> listeners = new ArrayList<RequestFutureListener<T>>();
-
-
-    /**
-     * Check whether the response is ready to be handled
-     * @return true if the response is ready, false otherwise
-     */
-    public boolean isDone() {
-        return isDone;
-    }
-
-    /**
-     * Get the value corresponding to this request (only available if the request succeeded)
-     * @return the value if it exists or null
-     */
-    public T value() {
-        return value;
-    }
-
-    /**
-     * Check if the request succeeded;
-     * @return true if the request completed and was successful
-     */
-    public boolean succeeded() {
-        return isDone && exception == null;
-    }
-
-    /**
-     * Check if the request failed.
-     * @return true if the request completed with a failure
-     */
-    public boolean failed() {
-        return isDone && exception != null;
-    }
-
-    /**
-     * Check if the request is retriable (convenience method for checking if
-     * the exception is an instance of {@link RetriableException}.
-     * @return true if it is retriable, false otherwise
-     */
-    public boolean isRetriable() {
-        return exception instanceof RetriableException;
-    }
-
-    /**
-     * Get the exception from a failed result (only available if the request failed)
-     * @return The exception if it exists or null
-     */
-    public RuntimeException exception() {
-        return exception;
-    }
-
-    /**
-     * Complete the request successfully. After this call, {@link #succeeded()} will return true
-     * and the value can be obtained through {@link #value()}.
-     * @param value corresponding value (or null if there is none)
-     */
-    public void complete(T value) {
-        this.value = value;
-        this.isDone = true;
-        fireSuccess();
-    }
-
-    /**
-     * Raise an exception. The request will be marked as failed, and the caller can either
-     * handle the exception or throw it.
-     * @param e corresponding exception to be passed to caller
-     */
-    public void raise(RuntimeException e) {
-        this.exception = e;
-        this.isDone = true;
-        fireFailure();
-    }
-
-    /**
-     * Raise an error. The request will be marked as failed.
-     * @param error corresponding error to be passed to caller
-     */
-    public void raise(Errors error) {
-        raise(error.exception());
-    }
-
-    private void fireSuccess() {
-        for (RequestFutureListener listener: listeners)
-            listener.onSuccess(value);
-    }
-
-    private void fireFailure() {
-        for (RequestFutureListener listener: listeners)
-            listener.onFailure(exception);
-    }
-
-    /**
-     * Add a listener which will be notified when the future completes
-     * @param listener
-     */
-    public void addListener(RequestFutureListener<T> listener) {
-        if (isDone) {
-            if (exception != null)
-                listener.onFailure(exception);
-            else
-                listener.onSuccess(value);
-        } else {
-            this.listeners.add(listener);
-        }
-    }
-
-    /**
-     * Convert from a request future of one type to another type
-     * @param adapter The adapter which does the conversion
-     * @param <S> The type of the future adapted to
-     * @return The new future
-     */
-    public <S> RequestFuture<S> compose(final RequestFutureAdapter<T, S> adapter) {
-        final RequestFuture<S> adapted = new RequestFuture<S>();
-        addListener(new RequestFutureListener<T>() {
-            @Override
-            public void onSuccess(T value) {
-                adapter.onSuccess(value, adapted);
-            }
-
-            @Override
-            public void onFailure(RuntimeException e) {
-                adapter.onFailure(e, adapted);
-            }
-        });
-        return adapted;
-    }
-
-    public static <T> RequestFuture<T> failure(RuntimeException e) {
-        RequestFuture<T> future = new RequestFuture<T>();
-        future.raise(e);
-        return future;
-    }
-
-    public static RequestFuture<Void> voidSuccess() {
-        RequestFuture<Void> future = new RequestFuture<Void>();
-        future.complete(null);
-        return future;
-    }
-
-    public static <T> RequestFuture<T> coordinatorNotAvailable() {
-        return failure(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception());
-    }
-
-    public static <T> RequestFuture<T> leaderNotAvailable() {
-        return failure(Errors.LEADER_NOT_AVAILABLE.exception());
-    }
-
-    public static <T> RequestFuture<T> noBrokersAvailable() {
-        return failure(new NoAvailableBrokersException());
-    }
-
-    public static <T> RequestFuture<T> staleMetadata() {
-        return failure(new StaleMetadataException());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFutureAdapter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFutureAdapter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFutureAdapter.java
deleted file mode 100644
index 0ed1231..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFutureAdapter.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.kafka.copied.clients.consumer.internals;
-
-/**
- * Adapt from a request future of one type to another.
- *
- * @param <F> Type to adapt from
- * @param <T> Type to adapt to
- */
-public abstract class RequestFutureAdapter<F, T> {
-
-    public abstract void onSuccess(F value, RequestFuture<T> future);
-
-    public void onFailure(RuntimeException e, RequestFuture<T> future) {
-        future.raise(e);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFutureListener.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFutureListener.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFutureListener.java
deleted file mode 100644
index 19ce5f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/RequestFutureListener.java
+++ /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.
- */
-package org.apache.kafka.copied.clients.consumer.internals;
-
-/**
- * Listener interface to hook into RequestFuture completion.
- */
-public interface RequestFutureListener<T> {
-
-    void onSuccess(T value);
-
-    void onFailure(RuntimeException e);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/SendFailedException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/SendFailedException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/SendFailedException.java
deleted file mode 100644
index d07fa5c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/SendFailedException.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer.internals;
-
-import org.apache.kafka.copied.common.errors.RetriableException;
-
-/**
- * Exception used in {@link ConsumerNetworkClient} to indicate the failure
- * to transmit a request to the networking layer. This could be either because
- * the client is still connecting to the given host or its send buffer is full.
- */
-public class SendFailedException extends RetriableException {
-    public static final SendFailedException INSTANCE = new SendFailedException();
-
-    private static final long serialVersionUID = 1L;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/StaleMetadataException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/StaleMetadataException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/StaleMetadataException.java
deleted file mode 100644
index 90faed7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/StaleMetadataException.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer.internals;
-
-import org.apache.kafka.copied.common.errors.InvalidMetadataException;
-
-/**
- * Thrown when metadata is old and needs to be refreshed.
- */
-public class StaleMetadataException extends InvalidMetadataException {
-    private static final long serialVersionUID = 1L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/SubscriptionState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/SubscriptionState.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/SubscriptionState.java
deleted file mode 100644
index c68895c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/SubscriptionState.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.kafka.copied.clients.consumer.internals;
-
-import org.apache.kafka.copied.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.copied.common.TopicPartition;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A class for tracking the topics, partitions, and offsets for the consumer
- */
-public class SubscriptionState {
-
-    /* the list of topics the user has requested */
-    private final Set<String> subscribedTopics;
-
-    /* the list of partitions the user has requested */
-    private final Set<TopicPartition> subscribedPartitions;
-
-    /* the list of partitions currently assigned */
-    private final Set<TopicPartition> assignedPartitions;
-
-    /* the offset exposed to the user */
-    private final Map<TopicPartition, Long> consumed;
-
-    /* the current point we have fetched up to */
-    private final Map<TopicPartition, Long> fetched;
-
-    /* the last committed offset for each partition */
-    private final Map<TopicPartition, Long> committed;
-
-    /* do we need to request a partition assignment from the coordinator? */
-    private boolean needsPartitionAssignment;
-
-    /* do we need to request the latest committed offsets from the coordinator? */
-    private boolean needsFetchCommittedOffsets;
-
-    /* Partitions that need to be reset before fetching */
-    private Map<TopicPartition, OffsetResetStrategy> resetPartitions;
-
-    /* Default offset reset strategy */
-    private OffsetResetStrategy offsetResetStrategy;
-
-    public SubscriptionState(OffsetResetStrategy offsetResetStrategy) {
-        this.offsetResetStrategy = offsetResetStrategy;
-        this.subscribedTopics = new HashSet<String>();
-        this.subscribedPartitions = new HashSet<TopicPartition>();
-        this.assignedPartitions = new HashSet<TopicPartition>();
-        this.consumed = new HashMap<TopicPartition, Long>();
-        this.fetched = new HashMap<TopicPartition, Long>();
-        this.committed = new HashMap<TopicPartition, Long>();
-        this.needsPartitionAssignment = false;
-        this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up
-        this.resetPartitions = new HashMap<TopicPartition, OffsetResetStrategy>();
-    }
-
-    public void subscribe(String topic) {
-        if (this.subscribedPartitions.size() > 0)
-            throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
-        if (!this.subscribedTopics.contains(topic)) {
-            this.subscribedTopics.add(topic);
-            this.needsPartitionAssignment = true;
-        }
-    }
-
-    public void unsubscribe(String topic) {
-        if (!this.subscribedTopics.contains(topic))
-            throw new IllegalStateException("Topic " + topic + " was never subscribed to.");
-        this.subscribedTopics.remove(topic);
-        this.needsPartitionAssignment = true;
-        for (TopicPartition tp: assignedPartitions())
-            if (topic.equals(tp.topic()))
-                clearPartition(tp);
-    }
-
-    public void needReassignment() {
-        this.needsPartitionAssignment = true;
-    }
-
-    public void subscribe(TopicPartition tp) {
-        if (this.subscribedTopics.size() > 0)
-            throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
-        this.subscribedPartitions.add(tp);
-        this.assignedPartitions.add(tp);
-    }
-
-    public void unsubscribe(TopicPartition partition) {
-        if (!subscribedPartitions.contains(partition))
-            throw new IllegalStateException("Partition " + partition + " was never subscribed to.");
-        subscribedPartitions.remove(partition);
-        clearPartition(partition);
-    }
-    
-    private void clearPartition(TopicPartition tp) {
-        this.assignedPartitions.remove(tp);
-        this.committed.remove(tp);
-        this.fetched.remove(tp);
-        this.consumed.remove(tp);
-        this.resetPartitions.remove(tp);
-    }
-
-    public void clearAssignment() {
-        this.assignedPartitions.clear();
-        this.committed.clear();
-        this.fetched.clear();
-        this.consumed.clear();
-        this.needsPartitionAssignment = !subscribedTopics().isEmpty();
-    }
-
-    public Set<String> subscribedTopics() {
-        return this.subscribedTopics;
-    }
-
-    public Long fetched(TopicPartition tp) {
-        return this.fetched.get(tp);
-    }
-
-    public void fetched(TopicPartition tp, long offset) {
-        if (!this.assignedPartitions.contains(tp))
-            throw new IllegalArgumentException("Can't change the fetch position for a partition you are not currently subscribed to.");
-        this.fetched.put(tp, offset);
-    }
-
-    public void committed(TopicPartition tp, long offset) {
-        this.committed.put(tp, offset);
-    }
-
-    public Long committed(TopicPartition tp) {
-        return this.committed.get(tp);
-    }
-
-    public void needRefreshCommits() {
-        this.needsFetchCommittedOffsets = true;
-    }
-
-    public boolean refreshCommitsNeeded() {
-        return this.needsFetchCommittedOffsets;
-    }
-
-    public void commitsRefreshed() {
-        this.needsFetchCommittedOffsets = false;
-    }
-    
-    public void seek(TopicPartition tp, long offset) {
-        fetched(tp, offset);
-        consumed(tp, offset);
-        resetPartitions.remove(tp);
-    }
-
-    public Set<TopicPartition> assignedPartitions() {
-        return this.assignedPartitions;
-    }
-
-    public boolean partitionsAutoAssigned() {
-        return !this.subscribedTopics.isEmpty();
-    }
-
-    public void consumed(TopicPartition tp, long offset) {
-        if (!this.assignedPartitions.contains(tp))
-            throw new IllegalArgumentException("Can't change the consumed position for a partition you are not currently subscribed to.");
-        this.consumed.put(tp, offset);
-    }
-
-    public Long consumed(TopicPartition partition) {
-        return this.consumed.get(partition);
-    }
-
-    public Map<TopicPartition, Long> allConsumed() {
-        return this.consumed;
-    }
-
-    public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) {
-        this.resetPartitions.put(partition, offsetResetStrategy);
-        this.fetched.remove(partition);
-        this.consumed.remove(partition);
-    }
-
-    public void needOffsetReset(TopicPartition partition) {
-        needOffsetReset(partition, offsetResetStrategy);
-    }
-
-    public boolean isOffsetResetNeeded(TopicPartition partition) {
-        return resetPartitions.containsKey(partition);
-    }
-
-    public boolean isOffsetResetNeeded() {
-        return !resetPartitions.isEmpty();
-    }
-
-    public OffsetResetStrategy resetStrategy(TopicPartition partition) {
-        return resetPartitions.get(partition);
-    }
-
-    public boolean hasAllFetchPositions() {
-        return this.fetched.size() >= this.assignedPartitions.size();
-    }
-
-    public Set<TopicPartition> missingFetchPositions() {
-        Set<TopicPartition> copy = new HashSet<TopicPartition>(this.assignedPartitions);
-        copy.removeAll(this.fetched.keySet());
-        return copy;
-    }
-
-    public boolean partitionAssignmentNeeded() {
-        return this.needsPartitionAssignment;
-    }
-
-    public void changePartitionAssignment(List<TopicPartition> assignments) {
-        for (TopicPartition tp : assignments)
-            if (!this.subscribedTopics.contains(tp.topic()))
-                throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic.");
-        this.clearAssignment();
-        this.assignedPartitions.addAll(assignments);
-        this.needsPartitionAssignment = false;
-    }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Cluster.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Cluster.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Cluster.java
deleted file mode 100644
index d6841eb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Cluster.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common;
-
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A representation of a subset of the nodes, topics, and partitions in the Kafka cluster.
- */
-public final class Cluster {
-
-    private final List<Node> nodes;
-    private final Map<TopicPartition, PartitionInfo> partitionsByTopicPartition;
-    private final Map<String, List<PartitionInfo>> partitionsByTopic;
-    private final Map<String, List<PartitionInfo>> availablePartitionsByTopic;
-    private final Map<Integer, List<PartitionInfo>> partitionsByNode;
-    private final Map<Integer, Node> nodesById;
-
-    /**
-     * Create a new cluster with the given nodes and partitions
-     * @param nodes The nodes in the cluster
-     * @param partitions Information about a subset of the topic-partitions this cluster hosts
-     */
-    public Cluster(Collection<Node> nodes, Collection<PartitionInfo> partitions) {
-        // make a randomized, unmodifiable copy of the nodes
-        List<Node> copy = new ArrayList<Node>(nodes);
-        Collections.shuffle(copy);
-        this.nodes = Collections.unmodifiableList(copy);
-        
-        this.nodesById = new HashMap<Integer, Node>();
-        for (Node node: nodes)
-            this.nodesById.put(node.id(), node);
-
-        // index the partitions by topic/partition for quick lookup
-        this.partitionsByTopicPartition = new HashMap<TopicPartition, PartitionInfo>(partitions.size());
-        for (PartitionInfo p : partitions)
-            this.partitionsByTopicPartition.put(new TopicPartition(p.topic(), p.partition()), p);
-
-        // index the partitions by topic and node respectively, and make the lists
-        // unmodifiable so we can hand them out in user-facing apis without risk
-        // of the client modifying the contents
-        HashMap<String, List<PartitionInfo>> partsForTopic = new HashMap<String, List<PartitionInfo>>();
-        HashMap<Integer, List<PartitionInfo>> partsForNode = new HashMap<Integer, List<PartitionInfo>>();
-        for (Node n : this.nodes) {
-            partsForNode.put(n.id(), new ArrayList<PartitionInfo>());
-        }
-        for (PartitionInfo p : partitions) {
-            if (!partsForTopic.containsKey(p.topic()))
-                partsForTopic.put(p.topic(), new ArrayList<PartitionInfo>());
-            List<PartitionInfo> psTopic = partsForTopic.get(p.topic());
-            psTopic.add(p);
-
-            if (p.leader() != null) {
-                List<PartitionInfo> psNode = Utils.notNull(partsForNode.get(p.leader().id()));
-                psNode.add(p);
-            }
-        }
-        this.partitionsByTopic = new HashMap<String, List<PartitionInfo>>(partsForTopic.size());
-        this.availablePartitionsByTopic = new HashMap<String, List<PartitionInfo>>(partsForTopic.size());
-        for (Map.Entry<String, List<PartitionInfo>> entry : partsForTopic.entrySet()) {
-            String topic = entry.getKey();
-            List<PartitionInfo> partitionList = entry.getValue();
-            this.partitionsByTopic.put(topic, Collections.unmodifiableList(partitionList));
-            List<PartitionInfo> availablePartitions = new ArrayList<PartitionInfo>();
-            for (PartitionInfo part : partitionList) {
-                if (part.leader() != null)
-                    availablePartitions.add(part);
-            }
-            this.availablePartitionsByTopic.put(topic, Collections.unmodifiableList(availablePartitions));
-        }
-        this.partitionsByNode = new HashMap<Integer, List<PartitionInfo>>(partsForNode.size());
-        for (Map.Entry<Integer, List<PartitionInfo>> entry : partsForNode.entrySet())
-            this.partitionsByNode.put(entry.getKey(), Collections.unmodifiableList(entry.getValue()));
-
-    }
-
-    /**
-     * Create an empty cluster instance with no nodes and no topic-partitions.
-     */
-    public static Cluster empty() {
-        return new Cluster(new ArrayList<Node>(0), new ArrayList<PartitionInfo>(0));
-    }
-
-    /**
-     * Create a "bootstrap" cluster using the given list of host/ports
-     * @param addresses The addresses
-     * @return A cluster for these hosts/ports
-     */
-    public static Cluster bootstrap(List<InetSocketAddress> addresses) {
-        List<Node> nodes = new ArrayList<Node>();
-        int nodeId = -1;
-        for (InetSocketAddress address : addresses)
-            nodes.add(new Node(nodeId--, address.getHostName(), address.getPort()));
-        return new Cluster(nodes, new ArrayList<PartitionInfo>(0));
-    }
-
-    /**
-     * @return The known set of nodes
-     */
-    public List<Node> nodes() {
-        return this.nodes;
-    }
-    
-    /**
-     * Get the node by the node id (or null if no such node exists)
-     * @param id The id of the node
-     * @return The node, or null if no such node exists
-     */
-    public Node nodeById(int id) {
-        return this.nodesById.get(id);
-    }
-
-    /**
-     * Get the current leader for the given topic-partition
-     * @param topicPartition The topic and partition we want to know the leader for
-     * @return The node that is the leader for this topic-partition, or null if there is currently no leader
-     */
-    public Node leaderFor(TopicPartition topicPartition) {
-        PartitionInfo info = partitionsByTopicPartition.get(topicPartition);
-        if (info == null)
-            return null;
-        else
-            return info.leader();
-    }
-
-    /**
-     * Get the metadata for the specified partition
-     * @param topicPartition The topic and partition to fetch info for
-     * @return The metadata about the given topic and partition
-     */
-    public PartitionInfo partition(TopicPartition topicPartition) {
-        return partitionsByTopicPartition.get(topicPartition);
-    }
-
-    /**
-     * Get the list of partitions for this topic
-     * @param topic The topic name
-     * @return A list of partitions
-     */
-    public List<PartitionInfo> partitionsForTopic(String topic) {
-        return this.partitionsByTopic.get(topic);
-    }
-
-    /**
-     * Get the list of available partitions for this topic
-     * @param topic The topic name
-     * @return A list of partitions
-     */
-    public List<PartitionInfo> availablePartitionsForTopic(String topic) {
-        return this.availablePartitionsByTopic.get(topic);
-    }
-
-    /**
-     * Get the list of partitions whose leader is this node
-     * @param nodeId The node id
-     * @return A list of partitions
-     */
-    public List<PartitionInfo> partitionsForNode(int nodeId) {
-        return this.partitionsByNode.get(nodeId);
-    }
-
-    /**
-     * Get all topics.
-     * @return a set of all topics
-     */
-    public Set<String> topics() {
-        return this.partitionsByTopic.keySet();
-    }
-
-    @Override
-    public String toString() {
-        return "Cluster(nodes = " + this.nodes + ", partitions = " + this.partitionsByTopicPartition.values() + ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Configurable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Configurable.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Configurable.java
deleted file mode 100644
index 04594f6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Configurable.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common;
-
-import java.util.Map;
-
-/**
- * A Mix-in style interface for classes that are instantiated by reflection and need to take configuration parameters
- */
-public interface Configurable {
-
-    /**
-     * Configure this class with the given key-value pairs
-     */
-    public void configure(Map<String, ?> configs);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/KafkaException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/KafkaException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/KafkaException.java
deleted file mode 100644
index bebac2c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/KafkaException.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common;
-
-/**
- * The base class of all other Kafka exceptions
- */
-public class KafkaException extends RuntimeException {
-
-    private final static long serialVersionUID = 1L;
-
-    public KafkaException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public KafkaException(String message) {
-        super(message);
-    }
-
-    public KafkaException(Throwable cause) {
-        super(cause);
-    }
-
-    public KafkaException() {
-        super();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Metric.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Metric.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Metric.java
deleted file mode 100644
index ed299e8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Metric.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.kafka.copied.common;
-
-/**
- * A numerical metric tracked for monitoring purposes
- */
-public interface Metric {
-
-    /**
-     * A name for this metric
-     */
-    public MetricName metricName();
-
-    /**
-     * The value of the metric
-     */
-    public double value();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/MetricName.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/MetricName.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/MetricName.java
deleted file mode 100644
index cb3044b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/MetricName.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common;
-
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The <code>MetricName</code> class encapsulates a metric's name, logical group and its related attributes
- * <p>
- * This class captures the following parameters
- * <pre>
- *  <b>name</b> The name of the metric
- *  <b>group</b> logical group name of the metrics to which this metric belongs.
- *  <b>description</b> A human-readable description to include in the metric. This is optional.
- *  <b>tags</b> additional key/value attributes of the metric. This is optional.
- * </pre>
- * group, tags parameters can be used to create unique metric names while reporting in JMX or any custom reporting.
- * <p>
- * Ex: standard JMX MBean can be constructed like  <b>domainName:type=group,key1=val1,key2=val2</b>
- * <p>
- * Usage looks something like this:
- * <pre>{@code
- * // set up metrics:
- * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
- * Sensor sensor = metrics.sensor("message-sizes");
- *
- * Map<String, String> metricTags = new LinkedHashMap<String, String>();
- * metricTags.put("client-id", "producer-1");
- * metricTags.put("topic", "topic");
- *
- * MetricName metricName = new MetricName("message-size-avg", "producer-metrics", "average message size", metricTags);
- * sensor.add(metricName, new Avg());
- *
- * metricName = new MetricName("message-size-max", "producer-metrics", metricTags);
- * sensor.add(metricName, new Max());
- *
- * metricName = new MetricName("message-size-min", "producer-metrics", "message minimum size", "client-id", "my-client", "topic", "my-topic");
- * sensor.add(metricName, new Min());
- *
- * // as messages are sent we record the sizes
- * sensor.record(messageSize);
- * }</pre>
- */
-public final class MetricName {
-
-    private final String name;
-    private final String group;
-    private final String description;
-    private Map<String, String> tags;
-    private int hash = 0;
-
-    /**
-     * @param name        The name of the metric
-     * @param group       logical group name of the metrics to which this metric belongs
-     * @param description A human-readable description to include in the metric
-     * @param tags        additional key/value attributes of the metric
-     */
-    public MetricName(String name, String group, String description, Map<String, String> tags) {
-        this.name = Utils.notNull(name);
-        this.group = Utils.notNull(group);
-        this.description = Utils.notNull(description);
-        this.tags = Utils.notNull(tags);
-    }
-
-    /**
-     * @param name          The name of the metric
-     * @param group         logical group name of the metrics to which this metric belongs
-     * @param description   A human-readable description to include in the metric
-     * @param keyValue      additional key/value attributes of the metric (must come in pairs)
-     */
-    public MetricName(String name, String group, String description, String... keyValue) {
-        this(name, group, description, getTags(keyValue));
-    }
-
-    private static Map<String, String> getTags(String... keyValue) {
-        if ((keyValue.length % 2) != 0)
-            throw new IllegalArgumentException("keyValue needs to be specified in paris");
-        Map<String, String> tags = new HashMap<String, String>();
-
-        for (int i = 0; i < keyValue.length / 2; i++)
-            tags.put(keyValue[i], keyValue[i + 1]);
-        return tags;
-    }
-
-    /**
-     * @param name  The name of the metric
-     * @param group logical group name of the metrics to which this metric belongs
-     * @param tags  key/value attributes of the metric
-     */
-    public MetricName(String name, String group, Map<String, String> tags) {
-        this(name, group, "", tags);
-    }
-
-    /**
-     * @param name        The name of the metric
-     * @param group       logical group name of the metrics to which this metric belongs
-     * @param description A human-readable description to include in the metric
-     */
-    public MetricName(String name, String group, String description) {
-        this(name, group, description, new HashMap<String, String>());
-    }
-
-    /**
-     * @param name  The name of the metric
-     * @param group logical group name of the metrics to which this metric belongs
-     */
-    public MetricName(String name, String group) {
-        this(name, group, "", new HashMap<String, String>());
-    }
-
-    public String name() {
-        return this.name;
-    }
-
-    public String group() {
-        return this.group;
-    }
-
-    public Map<String, String> tags() {
-        return this.tags;
-    }
-
-    public String description() {
-        return this.description;
-    }
-
-    @Override
-    public int hashCode() {
-        if (hash != 0)
-            return hash;
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((group == null) ? 0 : group.hashCode());
-        result = prime * result + ((name == null) ? 0 : name.hashCode());
-        result = prime * result + ((tags == null) ? 0 : tags.hashCode());
-        this.hash = result;
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        MetricName other = (MetricName) obj;
-        if (group == null) {
-            if (other.group != null)
-                return false;
-        } else if (!group.equals(other.group))
-            return false;
-        if (name == null) {
-            if (other.name != null)
-                return false;
-        } else if (!name.equals(other.name))
-            return false;
-        if (tags == null) {
-            if (other.tags != null)
-                return false;
-        } else if (!tags.equals(other.tags))
-            return false;
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        return "MetricName [name=" + name + ", group=" + group + ", description="
-                + description + ", tags=" + tags + "]";
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Node.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Node.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Node.java
deleted file mode 100644
index db006b3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/Node.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.kafka.copied.common;
-
-import java.io.Serializable;
-
-/**
- * Information about a Kafka node
- */
-public class Node implements Serializable {
-
-    private final int id;
-    private final String idString;
-    private final String host;
-    private final int port;
-
-    public Node(int id, String host, int port) {
-        super();
-        this.id = id;
-        this.idString = Integer.toString(id);
-        this.host = host;
-        this.port = port;
-    }
-
-    public static Node noNode() {
-        return new Node(-1, "", -1);
-    }
-
-    /**
-     * The node id of this node
-     */
-    public int id() {
-        return id;
-    }
-
-    /**
-     * String representation of the node id.
-     * Typically the integer id is used to serialize over the wire, the string representation is used as an identifier with NetworkClient code
-     */
-    public String idString() {
-        return idString;
-    }
-
-    /**
-     * The host name for this node
-     */
-    public String host() {
-        return host;
-    }
-
-    /**
-     * The port for this node
-     */
-    public int port() {
-        return port;
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((host == null) ? 0 : host.hashCode());
-        result = prime * result + id;
-        result = prime * result + port;
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        Node other = (Node) obj;
-        if (host == null) {
-            if (other.host != null)
-                return false;
-        } else if (!host.equals(other.host))
-            return false;
-        if (id != other.id)
-            return false;
-        if (port != other.port)
-            return false;
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        return "Node(" + id + ", " + host + ", " + port + ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/PartitionInfo.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/PartitionInfo.java
deleted file mode 100644
index 0480755..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/PartitionInfo.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common;
-
-/**
- * Information about a topic-partition.
- */
-public class PartitionInfo {
-
-    private final String topic;
-    private final int partition;
-    private final Node leader;
-    private final Node[] replicas;
-    private final Node[] inSyncReplicas;
-
-    public PartitionInfo(String topic, int partition, Node leader, Node[] replicas, Node[] inSyncReplicas) {
-        this.topic = topic;
-        this.partition = partition;
-        this.leader = leader;
-        this.replicas = replicas;
-        this.inSyncReplicas = inSyncReplicas;
-    }
-
-    /**
-     * The topic name
-     */
-    public String topic() {
-        return topic;
-    }
-
-    /**
-     * The partition id
-     */
-    public int partition() {
-        return partition;
-    }
-
-    /**
-     * The node id of the node currently acting as a leader for this partition or -1 if there is no leader
-     */
-    public Node leader() {
-        return leader;
-    }
-
-    /**
-     * The complete set of replicas for this partition regardless of whether they are alive or up-to-date
-     */
-    public Node[] replicas() {
-        return replicas;
-    }
-
-    /**
-     * The subset of the replicas that are in sync, that is caught-up to the leader and ready to take over as leader if
-     * the leader should fail
-     */
-    public Node[] inSyncReplicas() {
-        return inSyncReplicas;
-    }
-
-    @Override
-    public String toString() {
-        return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s",
-                topic,
-                partition,
-                leader == null ? "none" : leader.id(),
-                fmtNodeIds(replicas),
-                fmtNodeIds(inSyncReplicas));
-    }
-
-    /* Extract the node ids from each item in the array and format for display */
-    private String fmtNodeIds(Node[] nodes) {
-        StringBuilder b = new StringBuilder("[");
-        for (int i = 0; i < nodes.length - 1; i++) {
-            b.append(Integer.toString(nodes[i].id()));
-            b.append(',');
-        }
-        if (nodes.length > 0) {
-            b.append(Integer.toString(nodes[nodes.length - 1].id()));
-            b.append(',');
-        }
-        b.append("]");
-        return b.toString();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/TopicPartition.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/TopicPartition.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/TopicPartition.java
deleted file mode 100644
index b7b5ba8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/TopicPartition.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common;
-
-import java.io.Serializable;
-
-/**
- * A topic name and partition number
- */
-public final class TopicPartition implements Serializable {
-
-    private int hash = 0;
-    private final int partition;
-    private final String topic;
-
-    public TopicPartition(String topic, int partition) {
-        this.partition = partition;
-        this.topic = topic;
-    }
-
-    public int partition() {
-        return partition;
-    }
-
-    public String topic() {
-        return topic;
-    }
-
-    @Override
-    public int hashCode() {
-        if (hash != 0)
-            return hash;
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + partition;
-        result = prime * result + ((topic == null) ? 0 : topic.hashCode());
-        this.hash = result;
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        TopicPartition other = (TopicPartition) obj;
-        if (partition != other.partition)
-            return false;
-        if (topic == null) {
-            if (other.topic != null)
-                return false;
-        } else if (!topic.equals(other.topic))
-            return false;
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        return topic + "-" + partition;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/AbstractConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/AbstractConfig.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/AbstractConfig.java
deleted file mode 100644
index 5425c13..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/AbstractConfig.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.kafka.copied.common.config;
-
-import org.apache.kafka.copied.common.Configurable;
-import org.apache.kafka.copied.common.KafkaException;
-import org.apache.kafka.copied.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A convenient base class for configurations to extend.
- * <p>
- * This class holds both the original configuration that was provided as well as the parsed
- */
-public class AbstractConfig {
-
-    private final Logger log = LoggerFactory.getLogger(getClass());
-
-    /* configs for which values have been requested, used to detect unused configs */
-    private final Set<String> used;
-
-    /* the original values passed in by the user */
-    private final Map<String, ?> originals;
-
-    /* the parsed values */
-    private final Map<String, Object> values;
-
-    @SuppressWarnings("unchecked")
-    public AbstractConfig(ConfigDef definition, Map<?, ?> originals) {
-        /* check that all the keys are really strings */
-        for (Object key : originals.keySet())
-            if (!(key instanceof String))
-                throw new ConfigException(key.toString(), originals.get(key), "Key must be a string.");
-        this.originals = (Map<String, ?>) originals;
-        this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<String>());
-        logAll();
-    }
-
-    protected Object get(String key) {
-        if (!values.containsKey(key))
-            throw new ConfigException(String.format("Unknown configuration '%s'", key));
-        used.add(key);
-        return values.get(key);
-    }
-
-    public Short getShort(String key) {
-        return (Short) get(key);
-    }
-
-    public Integer getInt(String key) {
-        return (Integer) get(key);
-    }
-
-    public Long getLong(String key) {
-        return (Long) get(key);
-    }
-
-    public Double getDouble(String key) {
-        return (Double) get(key);
-    }
-
-    @SuppressWarnings("unchecked")
-    public List<String> getList(String key) {
-        return (List<String>) get(key);
-    }
-
-    public boolean getBoolean(String key) {
-        return (Boolean) get(key);
-    }
-
-    public String getString(String key) {
-        return (String) get(key);
-    }
-
-    public Class<?> getClass(String key) {
-        return (Class<?>) get(key);
-    }
-
-    public Set<String> unused() {
-        Set<String> keys = new HashSet<String>(originals.keySet());
-        keys.removeAll(used);
-        return keys;
-    }
-
-    public Map<String, Object> originals() {
-        Map<String, Object> copy = new HashMap<String, Object>();
-        copy.putAll(originals);
-        return copy;
-    }
-
-    private void logAll() {
-        StringBuilder b = new StringBuilder();
-        b.append(getClass().getSimpleName());
-        b.append(" values: ");
-        b.append(Utils.NL);
-        for (Map.Entry<String, Object> entry : this.values.entrySet()) {
-            b.append('\t');
-            b.append(entry.getKey());
-            b.append(" = ");
-            b.append(entry.getValue());
-            b.append(Utils.NL);
-        }
-        log.info(b.toString());
-    }
-
-    /**
-     * Log warnings for any unused configurations
-     */
-    public void logUnused() {
-        for (String key : unused())
-            log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.values.get(key));
-    }
-
-    /**
-     * Get a configured instance of the give class specified by the given configuration key. If the object implements
-     * Configurable configure it using the configuration.
-     * 
-     * @param key The configuration key for the class
-     * @param t The interface the class should implement
-     * @return A configured instance of the class
-     */
-    public <T> T getConfiguredInstance(String key, Class<T> t) {
-        Class<?> c = getClass(key);
-        if (c == null)
-            return null;
-        Object o = Utils.newInstance(c);
-        if (!t.isInstance(o))
-            throw new KafkaException(c.getName() + " is not an instance of " + t.getName());
-        if (o instanceof Configurable)
-            ((Configurable) o).configure(this.originals);
-        return t.cast(o);
-    }
-
-    public <T> List<T> getConfiguredInstances(String key, Class<T> t) {
-        List<String> klasses = getList(key);
-        List<T> objects = new ArrayList<T>();
-        for (String klass : klasses) {
-            Class<?> c;
-            try {
-                c = Class.forName(klass);
-            } catch (ClassNotFoundException e) {
-                throw new ConfigException(key, klass, "Class " + klass + " could not be found.");
-            }
-            if (c == null)
-                return null;
-            Object o = Utils.newInstance(c);
-            if (!t.isInstance(o))
-                throw new KafkaException(c.getName() + " is not an instance of " + t.getName());
-            if (o instanceof Configurable)
-                ((Configurable) o).configure(this.originals);
-            objects.add(t.cast(o));
-        }
-        return objects;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/ConfigDef.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/ConfigDef.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/ConfigDef.java
deleted file mode 100644
index 5202c09..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/ConfigDef.java
+++ /dev/null
@@ -1,447 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.config;
-
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * This class is used for specifying the set of expected configurations, their type, their defaults, their
- * documentation, and any special validation logic used for checking the correctness of the values the user provides.
- * <p/>
- * Usage of this class looks something like this:
- * <p/>
- * <pre>
- * ConfigDef defs = new ConfigDef();
- * defs.define(&quot;config_name&quot;, Type.STRING, &quot;default string value&quot;, &quot;This configuration is used for blah blah blah.&quot;);
- * defs.define(&quot;another_config_name&quot;, Type.INT, 42, Range.atLeast(0), &quot;More documentation on this config&quot;);
- *
- * Properties props = new Properties();
- * props.setProperty(&quot;config_name&quot;, &quot;some value&quot;);
- * Map&lt;String, Object&gt; configs = defs.parse(props);
- *
- * String someConfig = (String) configs.get(&quot;config_name&quot;); // will return &quot;some value&quot;
- * int anotherConfig = (Integer) configs.get(&quot;another_config_name&quot;); // will return default value of 42
- * </pre>
- * <p/>
- * This class can be used stand-alone or in combination with {@link AbstractConfig} which provides some additional
- * functionality for accessing configs.
- */
-public class ConfigDef {
-
-    private static final Object NO_DEFAULT_VALUE = new String("");
-
-    private final Map<String, ConfigKey> configKeys = new HashMap<String, ConfigKey>();
-
-    /**
-     * Returns unmodifiable set of properties names defined in this {@linkplain ConfigDef}
-     *
-     * @return new unmodifiable {@link Set} instance containing the keys
-     */
-    public Set<String> names() {
-        return Collections.unmodifiableSet(configKeys.keySet());
-    }
-
-    /**
-     * Define a new configuration
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param defaultValue  The default value to use if this config isn't present
-     * @param validator     A validator to use in checking the correctness of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @param required      Should the config fail if given property is not set and doesn't have default value specified
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation,
-                            boolean required) {
-        if (configKeys.containsKey(name))
-            throw new ConfigException("Configuration " + name + " is defined twice.");
-        Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type);
-        configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation, required));
-        return this;
-    }
-
-    /**
-     * Define a new required configuration
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param defaultValue  The default value to use if this config isn't present
-     * @param validator     A validator to use in checking the correctness of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation) {
-        return define(name, type, defaultValue, validator, importance, documentation, true);
-    }
-
-    /**
-     * Define a new configuration with no special validation logic
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param defaultValue  The default value to use if this config isn't present
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Object defaultValue, Importance importance, String documentation) {
-        return define(name, type, defaultValue, null, importance, documentation, true);
-    }
-
-    /**
-     * Define a required parameter with no default value
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param validator     A validator to use in checking the correctness of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Validator validator, Importance importance, String documentation) {
-        return define(name, type, NO_DEFAULT_VALUE, validator, importance, documentation, true);
-    }
-
-    /**
-     * Define a required parameter with no default value and no special validation logic
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Importance importance, String documentation) {
-        return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, true);
-    }
-
-    /**
-     * Define a required parameter with no default value and no special validation logic
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @param required      Should the config fail if given property is not set and doesn't have default value specified
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Importance importance, String documentation, boolean required) {
-        return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, required);
-    }
-
-
-    /**
-     * Parse and validate configs against this configuration definition. The input is a map of configs. It is expected
-     * that the keys of the map are strings, but the values can either be strings or they may already be of the
-     * appropriate type (int, string, etc). This will work equally well with either java.util.Properties instances or a
-     * programmatically constructed map.
-     *
-     * @param props The configs to parse and validate
-     * @return Parsed and validated configs. The key will be the config name and the value will be the value parsed into
-     * the appropriate type (int, string, etc)
-     */
-    public Map<String, Object> parse(Map<?, ?> props) {
-        /* parse all known keys */
-        Map<String, Object> values = new HashMap<String, Object>();
-        for (ConfigKey key : configKeys.values()) {
-            Object value;
-            // props map contains setting - assign ConfigKey value
-            if (props.containsKey(key.name))
-                value = parseType(key.name, props.get(key.name), key.type);
-                // props map doesn't contain setting, the key is required and no default value specified - it's an error
-            else if (key.defaultValue == NO_DEFAULT_VALUE && key.required)
-                throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value.");
-                // props map doesn't contain setting, no default value specified and the key is not required - assign it to null
-            else if (!key.hasDefault() && !key.required)
-                value = null;
-                // otherwise assign setting it's default value
-            else
-                value = key.defaultValue;
-            if (key.validator != null)
-                key.validator.ensureValid(key.name, value);
-            values.put(key.name, value);
-        }
-        return values;
-    }
-
-    /**
-     * Parse a value according to its expected type.
-     *
-     * @param name  The config name
-     * @param value The config value
-     * @param type  The expected type
-     * @return The parsed object
-     */
-    private Object parseType(String name, Object value, Type type) {
-        try {
-            String trimmed = null;
-            if (value instanceof String)
-                trimmed = ((String) value).trim();
-            switch (type) {
-                case BOOLEAN:
-                    if (value instanceof String) {
-                        if (trimmed.equalsIgnoreCase("true"))
-                            return true;
-                        else if (trimmed.equalsIgnoreCase("false"))
-                            return false;
-                        else
-                            throw new ConfigException(name, value, "Expected value to be either true or false");
-                    } else if (value instanceof Boolean)
-                        return value;
-                    else
-                        throw new ConfigException(name, value, "Expected value to be either true or false");
-                case STRING:
-                    if (value instanceof String)
-                        return trimmed;
-                    else
-                        throw new ConfigException(name, value, "Expected value to be a string, but it was a " + value.getClass().getName());
-                case INT:
-                    if (value instanceof Integer) {
-                        return (Integer) value;
-                    } else if (value instanceof String) {
-                        return Integer.parseInt(trimmed);
-                    } else {
-                        throw new ConfigException(name, value, "Expected value to be an number.");
-                    }
-                case SHORT:
-                    if (value instanceof Short) {
-                        return (Short) value;
-                    } else if (value instanceof String) {
-                        return Short.parseShort(trimmed);
-                    } else {
-                        throw new ConfigException(name, value, "Expected value to be an number.");
-                    }
-                case LONG:
-                    if (value instanceof Integer)
-                        return ((Integer) value).longValue();
-                    if (value instanceof Long)
-                        return (Long) value;
-                    else if (value instanceof String)
-                        return Long.parseLong(trimmed);
-                    else
-                        throw new ConfigException(name, value, "Expected value to be an number.");
-                case DOUBLE:
-                    if (value instanceof Number)
-                        return ((Number) value).doubleValue();
-                    else if (value instanceof String)
-                        return Double.parseDouble(trimmed);
-                    else
-                        throw new ConfigException(name, value, "Expected value to be an number.");
-                case LIST:
-                    if (value instanceof List)
-                        return (List<?>) value;
-                    else if (value instanceof String)
-                        if (trimmed.isEmpty())
-                            return Collections.emptyList();
-                        else
-                            return Arrays.asList(trimmed.split("\\s*,\\s*", -1));
-                    else
-                        throw new ConfigException(name, value, "Expected a comma separated list.");
-                case CLASS:
-                    if (value instanceof Class)
-                        return (Class<?>) value;
-                    else if (value instanceof String)
-                        return Class.forName(trimmed);
-                    else
-                        throw new ConfigException(name, value, "Expected a Class instance or class name.");
-                default:
-                    throw new IllegalStateException("Unknown type.");
-            }
-        } catch (NumberFormatException e) {
-            throw new ConfigException(name, value, "Not a number of type " + type);
-        } catch (ClassNotFoundException e) {
-            throw new ConfigException(name, value, "Class " + value + " could not be found.");
-        }
-    }
-
-    /**
-     * The config types
-     */
-    public enum Type {
-        BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS;
-    }
-
-    public enum Importance {
-        HIGH, MEDIUM, LOW
-    }
-
-    /**
-     * Validation logic the user may provide
-     */
-    public interface Validator {
-        public void ensureValid(String name, Object o);
-    }
-
-    /**
-     * Validation logic for numeric ranges
-     */
-    public static class Range implements Validator {
-        private final Number min;
-        private final Number max;
-
-        private Range(Number min, Number max) {
-            this.min = min;
-            this.max = max;
-        }
-
-        /**
-         * A numeric range that checks only the lower bound
-         *
-         * @param min The minimum acceptable value
-         */
-        public static Range atLeast(Number min) {
-            return new Range(min, null);
-        }
-
-        /**
-         * A numeric range that checks both the upper and lower bound
-         */
-        public static Range between(Number min, Number max) {
-            return new Range(min, max);
-        }
-
-        public void ensureValid(String name, Object o) {
-            Number n = (Number) o;
-            if (min != null && n.doubleValue() < min.doubleValue())
-                throw new ConfigException(name, o, "Value must be at least " + min);
-            if (max != null && n.doubleValue() > max.doubleValue())
-                throw new ConfigException(name, o, "Value must be no more than " + max);
-        }
-
-        public String toString() {
-            if (min == null)
-                return "[...," + max + "]";
-            else if (max == null)
-                return "[" + min + ",...]";
-            else
-                return "[" + min + ",...," + max + "]";
-        }
-    }
-
-    public static class ValidString implements Validator {
-        List<String> validStrings;
-
-        private ValidString(List<String> validStrings) {
-            this.validStrings = validStrings;
-        }
-
-        public static ValidString in(String... validStrings) {
-            return new ValidString(Arrays.asList(validStrings));
-        }
-
-        @Override
-        public void ensureValid(String name, Object o) {
-            String s = (String) o;
-            if (!validStrings.contains(s)) {
-                throw new ConfigException(name, o, "String must be one of: " + Utils.join(validStrings, ", "));
-            }
-
-        }
-
-        public String toString() {
-            return "[" + Utils.join(validStrings, ", ") + "]";
-        }
-    }
-
-    private static class ConfigKey {
-        public final String name;
-        public final Type type;
-        public final String documentation;
-        public final Object defaultValue;
-        public final Validator validator;
-        public final Importance importance;
-        public final boolean required;
-
-        public ConfigKey(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, boolean required) {
-            super();
-            this.name = name;
-            this.type = type;
-            this.defaultValue = defaultValue;
-            this.validator = validator;
-            this.importance = importance;
-            if (this.validator != null)
-                this.validator.ensureValid(name, defaultValue);
-            this.documentation = documentation;
-            this.required = required;
-        }
-
-        public boolean hasDefault() {
-            return this.defaultValue != NO_DEFAULT_VALUE;
-        }
-
-    }
-
-    public String toHtmlTable() {
-        // sort first required fields, then by importance, then name
-        List<ConfigKey> configs = new ArrayList<ConfigKey>(this.configKeys.values());
-        Collections.sort(configs, new Comparator<ConfigKey>() {
-            public int compare(ConfigDef.ConfigKey k1, ConfigDef.ConfigKey k2) {
-                // first take anything with no default value
-                if (!k1.hasDefault() && k2.hasDefault())
-                    return -1;
-                else if (!k2.hasDefault() && k1.hasDefault())
-                    return 1;
-
-                // then sort by importance
-                int cmp = k1.importance.compareTo(k2.importance);
-                if (cmp == 0)
-                    // then sort in alphabetical order
-                    return k1.name.compareTo(k2.name);
-                else
-                    return cmp;
-            }
-        });
-        StringBuilder b = new StringBuilder();
-        b.append("<table>\n");
-        b.append("<tr>\n");
-        b.append("<th>Name</th>\n");
-        b.append("<th>Type</th>\n");
-        b.append("<th>Default</th>\n");
-        b.append("<th>Importance</th>\n");
-        b.append("<th>Description</th>\n");
-        b.append("</tr>\n");
-        for (ConfigKey def : configs) {
-            b.append("<tr>\n");
-            b.append("<td>");
-            b.append(def.name);
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.type.toString().toLowerCase());
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.defaultValue == null ? "" : def.defaultValue);
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.importance.toString().toLowerCase());
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.documentation);
-            b.append("</td>");
-            b.append("</tr>\n");
-        }
-        b.append("</table>");
-        return b.toString();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/ConfigException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/ConfigException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/ConfigException.java
deleted file mode 100644
index a03de82..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/config/ConfigException.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.kafka.copied.common.config;
-
-import org.apache.kafka.copied.common.KafkaException;
-
-/**
- * Thrown if the user supplies an invalid configuration
- */
-public class ConfigException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public ConfigException(String message) {
-        super(message);
-    }
-
-    public ConfigException(String name, Object value) {
-        this(name, value, null);
-    }
-
-    public ConfigException(String name, Object value, String message) {
-        super("Invalid value " + value + " for configuration " + name + (message == null ? "" : ": " + message));
-    }
-
-}


[46/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureListener.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureListener.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureListener.java
deleted file mode 100644
index 90a1cfa..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureListener.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.kafka_backport.clients.consumer.internals;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Listener interface to hook into RequestFuture completion.
- */
-public interface RequestFutureListener<T> {
-
-    void onSuccess(T value);
-
-    void onFailure(RuntimeException e);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SendFailedException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SendFailedException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SendFailedException.java
deleted file mode 100644
index d94486e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SendFailedException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer.internals;
-
-import org.apache.flink.kafka_backport.common.errors.RetriableException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Exception used in {@link ConsumerNetworkClient} to indicate the failure
- * to transmit a request to the networking layer. This could be either because
- * the client is still connecting to the given host or its send buffer is full.
- */
-public class SendFailedException extends RetriableException {
-    public static final SendFailedException INSTANCE = new SendFailedException();
-
-    private static final long serialVersionUID = 1L;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/StaleMetadataException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/StaleMetadataException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/StaleMetadataException.java
deleted file mode 100644
index adff6e0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/StaleMetadataException.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer.internals;
-
-import org.apache.flink.kafka_backport.common.errors.InvalidMetadataException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Thrown when metadata is old and needs to be refreshed.
- */
-public class StaleMetadataException extends InvalidMetadataException {
-    private static final long serialVersionUID = 1L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SubscriptionState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SubscriptionState.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SubscriptionState.java
deleted file mode 100644
index f5e8802..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SubscriptionState.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer.internals;
-
-import org.apache.flink.kafka_backport.clients.consumer.OffsetResetStrategy;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A class for tracking the topics, partitions, and offsets for the consumer
- */
-public class SubscriptionState {
-
-    /* the list of topics the user has requested */
-    private final Set<String> subscribedTopics;
-
-    /* the list of partitions the user has requested */
-    private final Set<TopicPartition> subscribedPartitions;
-
-    /* the list of partitions currently assigned */
-    private final Set<TopicPartition> assignedPartitions;
-
-    /* the offset exposed to the user */
-    private final Map<TopicPartition, Long> consumed;
-
-    /* the current point we have fetched up to */
-    private final Map<TopicPartition, Long> fetched;
-
-    /* the last committed offset for each partition */
-    private final Map<TopicPartition, Long> committed;
-
-    /* do we need to request a partition assignment from the coordinator? */
-    private boolean needsPartitionAssignment;
-
-    /* do we need to request the latest committed offsets from the coordinator? */
-    private boolean needsFetchCommittedOffsets;
-
-    /* Partitions that need to be reset before fetching */
-    private Map<TopicPartition, OffsetResetStrategy> resetPartitions;
-
-    /* Default offset reset strategy */
-    private OffsetResetStrategy offsetResetStrategy;
-
-    public SubscriptionState(OffsetResetStrategy offsetResetStrategy) {
-        this.offsetResetStrategy = offsetResetStrategy;
-        this.subscribedTopics = new HashSet<String>();
-        this.subscribedPartitions = new HashSet<TopicPartition>();
-        this.assignedPartitions = new HashSet<TopicPartition>();
-        this.consumed = new HashMap<TopicPartition, Long>();
-        this.fetched = new HashMap<TopicPartition, Long>();
-        this.committed = new HashMap<TopicPartition, Long>();
-        this.needsPartitionAssignment = false;
-        this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up
-        this.resetPartitions = new HashMap<TopicPartition, OffsetResetStrategy>();
-    }
-
-    public void subscribe(String topic) {
-        if (this.subscribedPartitions.size() > 0)
-            throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
-        if (!this.subscribedTopics.contains(topic)) {
-            this.subscribedTopics.add(topic);
-            this.needsPartitionAssignment = true;
-        }
-    }
-
-    public void unsubscribe(String topic) {
-        if (!this.subscribedTopics.contains(topic))
-            throw new IllegalStateException("Topic " + topic + " was never subscribed to.");
-        this.subscribedTopics.remove(topic);
-        this.needsPartitionAssignment = true;
-        for (TopicPartition tp: assignedPartitions())
-            if (topic.equals(tp.topic()))
-                clearPartition(tp);
-    }
-
-    public void needReassignment() {
-        this.needsPartitionAssignment = true;
-    }
-
-    public void subscribe(TopicPartition tp) {
-        if (this.subscribedTopics.size() > 0)
-            throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
-        this.subscribedPartitions.add(tp);
-        this.assignedPartitions.add(tp);
-    }
-
-    public void unsubscribe(TopicPartition partition) {
-        if (!subscribedPartitions.contains(partition))
-            throw new IllegalStateException("Partition " + partition + " was never subscribed to.");
-        subscribedPartitions.remove(partition);
-        clearPartition(partition);
-    }
-    
-    private void clearPartition(TopicPartition tp) {
-        this.assignedPartitions.remove(tp);
-        this.committed.remove(tp);
-        this.fetched.remove(tp);
-        this.consumed.remove(tp);
-        this.resetPartitions.remove(tp);
-    }
-
-    public void clearAssignment() {
-        this.assignedPartitions.clear();
-        this.committed.clear();
-        this.fetched.clear();
-        this.consumed.clear();
-        this.needsPartitionAssignment = !subscribedTopics().isEmpty();
-    }
-
-    public Set<String> subscribedTopics() {
-        return this.subscribedTopics;
-    }
-
-    public Long fetched(TopicPartition tp) {
-        return this.fetched.get(tp);
-    }
-
-    public void fetched(TopicPartition tp, long offset) {
-        if (!this.assignedPartitions.contains(tp))
-            throw new IllegalArgumentException("Can't change the fetch position for a partition you are not currently subscribed to.");
-        this.fetched.put(tp, offset);
-    }
-
-    public void committed(TopicPartition tp, long offset) {
-        this.committed.put(tp, offset);
-    }
-
-    public Long committed(TopicPartition tp) {
-        return this.committed.get(tp);
-    }
-
-    public void needRefreshCommits() {
-        this.needsFetchCommittedOffsets = true;
-    }
-
-    public boolean refreshCommitsNeeded() {
-        return this.needsFetchCommittedOffsets;
-    }
-
-    public void commitsRefreshed() {
-        this.needsFetchCommittedOffsets = false;
-    }
-    
-    public void seek(TopicPartition tp, long offset) {
-        fetched(tp, offset);
-        consumed(tp, offset);
-        resetPartitions.remove(tp);
-    }
-
-    public Set<TopicPartition> assignedPartitions() {
-        return this.assignedPartitions;
-    }
-
-    public boolean partitionsAutoAssigned() {
-        return !this.subscribedTopics.isEmpty();
-    }
-
-    public void consumed(TopicPartition tp, long offset) {
-        if (!this.assignedPartitions.contains(tp))
-            throw new IllegalArgumentException("Can't change the consumed position for a partition you are not currently subscribed to.");
-        this.consumed.put(tp, offset);
-    }
-
-    public Long consumed(TopicPartition partition) {
-        return this.consumed.get(partition);
-    }
-
-    public Map<TopicPartition, Long> allConsumed() {
-        return this.consumed;
-    }
-
-    public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) {
-        this.resetPartitions.put(partition, offsetResetStrategy);
-        this.fetched.remove(partition);
-        this.consumed.remove(partition);
-    }
-
-    public void needOffsetReset(TopicPartition partition) {
-        needOffsetReset(partition, offsetResetStrategy);
-    }
-
-    public boolean isOffsetResetNeeded(TopicPartition partition) {
-        return resetPartitions.containsKey(partition);
-    }
-
-    public boolean isOffsetResetNeeded() {
-        return !resetPartitions.isEmpty();
-    }
-
-    public OffsetResetStrategy resetStrategy(TopicPartition partition) {
-        return resetPartitions.get(partition);
-    }
-
-    public boolean hasAllFetchPositions() {
-        return this.fetched.size() >= this.assignedPartitions.size();
-    }
-
-    public Set<TopicPartition> missingFetchPositions() {
-        Set<TopicPartition> copy = new HashSet<TopicPartition>(this.assignedPartitions);
-        copy.removeAll(this.fetched.keySet());
-        return copy;
-    }
-
-    public boolean partitionAssignmentNeeded() {
-        return this.needsPartitionAssignment;
-    }
-
-    public void changePartitionAssignment(List<TopicPartition> assignments) {
-        for (TopicPartition tp : assignments)
-            if (!this.subscribedTopics.contains(tp.topic()))
-                throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic.");
-        this.clearAssignment();
-        this.assignedPartitions.addAll(assignments);
-        this.needsPartitionAssignment = false;
-    }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Cluster.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Cluster.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Cluster.java
deleted file mode 100644
index f5e12d3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Cluster.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.kafka_backport.common;
-
-import org.apache.flink.kafka_backport.common.utils.Utils;
-
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A representation of a subset of the nodes, topics, and partitions in the Kafka cluster.
- */
-public final class Cluster {
-
-    private final List<Node> nodes;
-    private final Map<TopicPartition, PartitionInfo> partitionsByTopicPartition;
-    private final Map<String, List<PartitionInfo>> partitionsByTopic;
-    private final Map<String, List<PartitionInfo>> availablePartitionsByTopic;
-    private final Map<Integer, List<PartitionInfo>> partitionsByNode;
-    private final Map<Integer, Node> nodesById;
-
-    /**
-     * Create a new cluster with the given nodes and partitions
-     * @param nodes The nodes in the cluster
-     * @param partitions Information about a subset of the topic-partitions this cluster hosts
-     */
-    public Cluster(Collection<Node> nodes, Collection<PartitionInfo> partitions) {
-        // make a randomized, unmodifiable copy of the nodes
-        List<Node> copy = new ArrayList<Node>(nodes);
-        Collections.shuffle(copy);
-        this.nodes = Collections.unmodifiableList(copy);
-        
-        this.nodesById = new HashMap<Integer, Node>();
-        for (Node node: nodes)
-            this.nodesById.put(node.id(), node);
-
-        // index the partitions by topic/partition for quick lookup
-        this.partitionsByTopicPartition = new HashMap<TopicPartition, PartitionInfo>(partitions.size());
-        for (PartitionInfo p : partitions)
-            this.partitionsByTopicPartition.put(new TopicPartition(p.topic(), p.partition()), p);
-
-        // index the partitions by topic and node respectively, and make the lists
-        // unmodifiable so we can hand them out in user-facing apis without risk
-        // of the client modifying the contents
-        HashMap<String, List<PartitionInfo>> partsForTopic = new HashMap<String, List<PartitionInfo>>();
-        HashMap<Integer, List<PartitionInfo>> partsForNode = new HashMap<Integer, List<PartitionInfo>>();
-        for (Node n : this.nodes) {
-            partsForNode.put(n.id(), new ArrayList<PartitionInfo>());
-        }
-        for (PartitionInfo p : partitions) {
-            if (!partsForTopic.containsKey(p.topic()))
-                partsForTopic.put(p.topic(), new ArrayList<PartitionInfo>());
-            List<PartitionInfo> psTopic = partsForTopic.get(p.topic());
-            psTopic.add(p);
-
-            if (p.leader() != null) {
-                List<PartitionInfo> psNode = Utils.notNull(partsForNode.get(p.leader().id()));
-                psNode.add(p);
-            }
-        }
-        this.partitionsByTopic = new HashMap<String, List<PartitionInfo>>(partsForTopic.size());
-        this.availablePartitionsByTopic = new HashMap<String, List<PartitionInfo>>(partsForTopic.size());
-        for (Map.Entry<String, List<PartitionInfo>> entry : partsForTopic.entrySet()) {
-            String topic = entry.getKey();
-            List<PartitionInfo> partitionList = entry.getValue();
-            this.partitionsByTopic.put(topic, Collections.unmodifiableList(partitionList));
-            List<PartitionInfo> availablePartitions = new ArrayList<PartitionInfo>();
-            for (PartitionInfo part : partitionList) {
-                if (part.leader() != null)
-                    availablePartitions.add(part);
-            }
-            this.availablePartitionsByTopic.put(topic, Collections.unmodifiableList(availablePartitions));
-        }
-        this.partitionsByNode = new HashMap<Integer, List<PartitionInfo>>(partsForNode.size());
-        for (Map.Entry<Integer, List<PartitionInfo>> entry : partsForNode.entrySet())
-            this.partitionsByNode.put(entry.getKey(), Collections.unmodifiableList(entry.getValue()));
-
-    }
-
-    /**
-     * Create an empty cluster instance with no nodes and no topic-partitions.
-     */
-    public static Cluster empty() {
-        return new Cluster(new ArrayList<Node>(0), new ArrayList<PartitionInfo>(0));
-    }
-
-    /**
-     * Create a "bootstrap" cluster using the given list of host/ports
-     * @param addresses The addresses
-     * @return A cluster for these hosts/ports
-     */
-    public static Cluster bootstrap(List<InetSocketAddress> addresses) {
-        List<Node> nodes = new ArrayList<Node>();
-        int nodeId = -1;
-        for (InetSocketAddress address : addresses)
-            nodes.add(new Node(nodeId--, address.getHostName(), address.getPort()));
-        return new Cluster(nodes, new ArrayList<PartitionInfo>(0));
-    }
-
-    /**
-     * @return The known set of nodes
-     */
-    public List<Node> nodes() {
-        return this.nodes;
-    }
-    
-    /**
-     * Get the node by the node id (or null if no such node exists)
-     * @param id The id of the node
-     * @return The node, or null if no such node exists
-     */
-    public Node nodeById(int id) {
-        return this.nodesById.get(id);
-    }
-
-    /**
-     * Get the current leader for the given topic-partition
-     * @param topicPartition The topic and partition we want to know the leader for
-     * @return The node that is the leader for this topic-partition, or null if there is currently no leader
-     */
-    public Node leaderFor(TopicPartition topicPartition) {
-        PartitionInfo info = partitionsByTopicPartition.get(topicPartition);
-        if (info == null)
-            return null;
-        else
-            return info.leader();
-    }
-
-    /**
-     * Get the metadata for the specified partition
-     * @param topicPartition The topic and partition to fetch info for
-     * @return The metadata about the given topic and partition
-     */
-    public PartitionInfo partition(TopicPartition topicPartition) {
-        return partitionsByTopicPartition.get(topicPartition);
-    }
-
-    /**
-     * Get the list of partitions for this topic
-     * @param topic The topic name
-     * @return A list of partitions
-     */
-    public List<PartitionInfo> partitionsForTopic(String topic) {
-        return this.partitionsByTopic.get(topic);
-    }
-
-    /**
-     * Get the list of available partitions for this topic
-     * @param topic The topic name
-     * @return A list of partitions
-     */
-    public List<PartitionInfo> availablePartitionsForTopic(String topic) {
-        return this.availablePartitionsByTopic.get(topic);
-    }
-
-    /**
-     * Get the list of partitions whose leader is this node
-     * @param nodeId The node id
-     * @return A list of partitions
-     */
-    public List<PartitionInfo> partitionsForNode(int nodeId) {
-        return this.partitionsByNode.get(nodeId);
-    }
-
-    /**
-     * Get all topics.
-     * @return a set of all topics
-     */
-    public Set<String> topics() {
-        return this.partitionsByTopic.keySet();
-    }
-
-    @Override
-    public String toString() {
-        return "Cluster(nodes = " + this.nodes + ", partitions = " + this.partitionsByTopicPartition.values() + ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Configurable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Configurable.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Configurable.java
deleted file mode 100644
index fef2136..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Configurable.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.kafka_backport.common;
-
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A Mix-in style interface for classes that are instantiated by reflection and need to take configuration parameters
- */
-public interface Configurable {
-
-    /**
-     * Configure this class with the given key-value pairs
-     */
-    public void configure(Map<String, ?> configs);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/KafkaException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/KafkaException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/KafkaException.java
deleted file mode 100644
index d9df6e8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/KafkaException.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.kafka_backport.common;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The base class of all other Kafka exceptions
- */
-public class KafkaException extends RuntimeException {
-
-    private final static long serialVersionUID = 1L;
-
-    public KafkaException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public KafkaException(String message) {
-        super(message);
-    }
-
-    public KafkaException(Throwable cause) {
-        super(cause);
-    }
-
-    public KafkaException() {
-        super();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Metric.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Metric.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Metric.java
deleted file mode 100644
index 8858ffe..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Metric.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.kafka_backport.common;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A numerical metric tracked for monitoring purposes
- */
-public interface Metric {
-
-    /**
-     * A name for this metric
-     */
-    public MetricName metricName();
-
-    /**
-     * The value of the metric
-     */
-    public double value();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/MetricName.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/MetricName.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/MetricName.java
deleted file mode 100644
index 18dd955..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/MetricName.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common;
-
-import org.apache.flink.kafka_backport.common.utils.Utils;
-
-import java.util.HashMap;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The <code>MetricName</code> class encapsulates a metric's name, logical group and its related attributes
- * <p>
- * This class captures the following parameters
- * <pre>
- *  <b>name</b> The name of the metric
- *  <b>group</b> logical group name of the metrics to which this metric belongs.
- *  <b>description</b> A human-readable description to include in the metric. This is optional.
- *  <b>tags</b> additional key/value attributes of the metric. This is optional.
- * </pre>
- * group, tags parameters can be used to create unique metric names while reporting in JMX or any custom reporting.
- * <p>
- * Ex: standard JMX MBean can be constructed like  <b>domainName:type=group,key1=val1,key2=val2</b>
- * <p>
- * Usage looks something like this:
- * <pre>{@code
- * // set up metrics:
- * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
- * Sensor sensor = metrics.sensor("message-sizes");
- *
- * Map<String, String> metricTags = new LinkedHashMap<String, String>();
- * metricTags.put("client-id", "producer-1");
- * metricTags.put("topic", "topic");
- *
- * MetricName metricName = new MetricName("message-size-avg", "producer-metrics", "average message size", metricTags);
- * sensor.add(metricName, new Avg());
- *
- * metricName = new MetricName("message-size-max", "producer-metrics", metricTags);
- * sensor.add(metricName, new Max());
- *
- * metricName = new MetricName("message-size-min", "producer-metrics", "message minimum size", "client-id", "my-client", "topic", "my-topic");
- * sensor.add(metricName, new Min());
- *
- * // as messages are sent we record the sizes
- * sensor.record(messageSize);
- * }</pre>
- */
-public final class MetricName {
-
-    private final String name;
-    private final String group;
-    private final String description;
-    private Map<String, String> tags;
-    private int hash = 0;
-
-    /**
-     * @param name        The name of the metric
-     * @param group       logical group name of the metrics to which this metric belongs
-     * @param description A human-readable description to include in the metric
-     * @param tags        additional key/value attributes of the metric
-     */
-    public MetricName(String name, String group, String description, Map<String, String> tags) {
-        this.name = Utils.notNull(name);
-        this.group = Utils.notNull(group);
-        this.description = Utils.notNull(description);
-        this.tags = Utils.notNull(tags);
-    }
-
-    /**
-     * @param name          The name of the metric
-     * @param group         logical group name of the metrics to which this metric belongs
-     * @param description   A human-readable description to include in the metric
-     * @param keyValue      additional key/value attributes of the metric (must come in pairs)
-     */
-    public MetricName(String name, String group, String description, String... keyValue) {
-        this(name, group, description, getTags(keyValue));
-    }
-
-    private static Map<String, String> getTags(String... keyValue) {
-        if ((keyValue.length % 2) != 0)
-            throw new IllegalArgumentException("keyValue needs to be specified in paris");
-        Map<String, String> tags = new HashMap<String, String>();
-
-        for (int i = 0; i < keyValue.length / 2; i++)
-            tags.put(keyValue[i], keyValue[i + 1]);
-        return tags;
-    }
-
-    /**
-     * @param name  The name of the metric
-     * @param group logical group name of the metrics to which this metric belongs
-     * @param tags  key/value attributes of the metric
-     */
-    public MetricName(String name, String group, Map<String, String> tags) {
-        this(name, group, "", tags);
-    }
-
-    /**
-     * @param name        The name of the metric
-     * @param group       logical group name of the metrics to which this metric belongs
-     * @param description A human-readable description to include in the metric
-     */
-    public MetricName(String name, String group, String description) {
-        this(name, group, description, new HashMap<String, String>());
-    }
-
-    /**
-     * @param name  The name of the metric
-     * @param group logical group name of the metrics to which this metric belongs
-     */
-    public MetricName(String name, String group) {
-        this(name, group, "", new HashMap<String, String>());
-    }
-
-    public String name() {
-        return this.name;
-    }
-
-    public String group() {
-        return this.group;
-    }
-
-    public Map<String, String> tags() {
-        return this.tags;
-    }
-
-    public String description() {
-        return this.description;
-    }
-
-    @Override
-    public int hashCode() {
-        if (hash != 0)
-            return hash;
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((group == null) ? 0 : group.hashCode());
-        result = prime * result + ((name == null) ? 0 : name.hashCode());
-        result = prime * result + ((tags == null) ? 0 : tags.hashCode());
-        this.hash = result;
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        MetricName other = (MetricName) obj;
-        if (group == null) {
-            if (other.group != null)
-                return false;
-        } else if (!group.equals(other.group))
-            return false;
-        if (name == null) {
-            if (other.name != null)
-                return false;
-        } else if (!name.equals(other.name))
-            return false;
-        if (tags == null) {
-            if (other.tags != null)
-                return false;
-        } else if (!tags.equals(other.tags))
-            return false;
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        return "MetricName [name=" + name + ", group=" + group + ", description="
-                + description + ", tags=" + tags + "]";
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Node.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Node.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Node.java
deleted file mode 100644
index dd0537e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Node.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.kafka_backport.common;
-
-import java.io.Serializable;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Information about a Kafka node
- */
-public class Node implements Serializable {
-
-    private final int id;
-    private final String idString;
-    private final String host;
-    private final int port;
-
-    public Node(int id, String host, int port) {
-        super();
-        this.id = id;
-        this.idString = Integer.toString(id);
-        this.host = host;
-        this.port = port;
-    }
-
-    public static Node noNode() {
-        return new Node(-1, "", -1);
-    }
-
-    /**
-     * The node id of this node
-     */
-    public int id() {
-        return id;
-    }
-
-    /**
-     * String representation of the node id.
-     * Typically the integer id is used to serialize over the wire, the string representation is used as an identifier with NetworkClient code
-     */
-    public String idString() {
-        return idString;
-    }
-
-    /**
-     * The host name for this node
-     */
-    public String host() {
-        return host;
-    }
-
-    /**
-     * The port for this node
-     */
-    public int port() {
-        return port;
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((host == null) ? 0 : host.hashCode());
-        result = prime * result + id;
-        result = prime * result + port;
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        Node other = (Node) obj;
-        if (host == null) {
-            if (other.host != null)
-                return false;
-        } else if (!host.equals(other.host))
-            return false;
-        if (id != other.id)
-            return false;
-        if (port != other.port)
-            return false;
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        return "Node(" + id + ", " + host + ", " + port + ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/PartitionInfo.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/PartitionInfo.java
deleted file mode 100644
index ac7cc61..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/PartitionInfo.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.kafka_backport.common;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Information about a topic-partition.
- */
-public class PartitionInfo {
-
-    private final String topic;
-    private final int partition;
-    private final Node leader;
-    private final Node[] replicas;
-    private final Node[] inSyncReplicas;
-
-    public PartitionInfo(String topic, int partition, Node leader, Node[] replicas, Node[] inSyncReplicas) {
-        this.topic = topic;
-        this.partition = partition;
-        this.leader = leader;
-        this.replicas = replicas;
-        this.inSyncReplicas = inSyncReplicas;
-    }
-
-    /**
-     * The topic name
-     */
-    public String topic() {
-        return topic;
-    }
-
-    /**
-     * The partition id
-     */
-    public int partition() {
-        return partition;
-    }
-
-    /**
-     * The node id of the node currently acting as a leader for this partition or -1 if there is no leader
-     */
-    public Node leader() {
-        return leader;
-    }
-
-    /**
-     * The complete set of replicas for this partition regardless of whether they are alive or up-to-date
-     */
-    public Node[] replicas() {
-        return replicas;
-    }
-
-    /**
-     * The subset of the replicas that are in sync, that is caught-up to the leader and ready to take over as leader if
-     * the leader should fail
-     */
-    public Node[] inSyncReplicas() {
-        return inSyncReplicas;
-    }
-
-    @Override
-    public String toString() {
-        return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s",
-                topic,
-                partition,
-                leader == null ? "none" : leader.id(),
-                fmtNodeIds(replicas),
-                fmtNodeIds(inSyncReplicas));
-    }
-
-    /* Extract the node ids from each item in the array and format for display */
-    private String fmtNodeIds(Node[] nodes) {
-        StringBuilder b = new StringBuilder("[");
-        for (int i = 0; i < nodes.length - 1; i++) {
-            b.append(Integer.toString(nodes[i].id()));
-            b.append(',');
-        }
-        if (nodes.length > 0) {
-            b.append(Integer.toString(nodes[nodes.length - 1].id()));
-            b.append(',');
-        }
-        b.append("]");
-        return b.toString();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/TopicPartition.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/TopicPartition.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/TopicPartition.java
deleted file mode 100644
index cfb4848..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/TopicPartition.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.kafka_backport.common;
-
-import java.io.Serializable;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A topic name and partition number
- */
-public final class TopicPartition implements Serializable {
-
-    private int hash = 0;
-    private final int partition;
-    private final String topic;
-
-    public TopicPartition(String topic, int partition) {
-        this.partition = partition;
-        this.topic = topic;
-    }
-
-    public int partition() {
-        return partition;
-    }
-
-    public String topic() {
-        return topic;
-    }
-
-    @Override
-    public int hashCode() {
-        if (hash != 0)
-            return hash;
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + partition;
-        result = prime * result + ((topic == null) ? 0 : topic.hashCode());
-        this.hash = result;
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        TopicPartition other = (TopicPartition) obj;
-        if (partition != other.partition)
-            return false;
-        if (topic == null) {
-            if (other.topic != null)
-                return false;
-        } else if (!topic.equals(other.topic))
-            return false;
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        return topic + "-" + partition;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/AbstractConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/AbstractConfig.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/AbstractConfig.java
deleted file mode 100644
index 1b5cbc9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/AbstractConfig.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.config;
-
-import org.apache.flink.kafka_backport.common.Configurable;
-import org.apache.flink.kafka_backport.common.KafkaException;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A convenient base class for configurations to extend.
- * <p>
- * This class holds both the original configuration that was provided as well as the parsed
- */
-public class AbstractConfig {
-
-    private final Logger log = LoggerFactory.getLogger(getClass());
-
-    /* configs for which values have been requested, used to detect unused configs */
-    private final Set<String> used;
-
-    /* the original values passed in by the user */
-    private final Map<String, ?> originals;
-
-    /* the parsed values */
-    private final Map<String, Object> values;
-
-    @SuppressWarnings("unchecked")
-    public AbstractConfig(ConfigDef definition, Map<?, ?> originals) {
-        /* check that all the keys are really strings */
-        for (Object key : originals.keySet())
-            if (!(key instanceof String))
-                throw new ConfigException(key.toString(), originals.get(key), "Key must be a string.");
-        this.originals = (Map<String, ?>) originals;
-        this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<String>());
-        logAll();
-    }
-
-    protected Object get(String key) {
-        if (!values.containsKey(key))
-            throw new ConfigException(String.format("Unknown configuration '%s'", key));
-        used.add(key);
-        return values.get(key);
-    }
-
-    public Short getShort(String key) {
-        return (Short) get(key);
-    }
-
-    public Integer getInt(String key) {
-        return (Integer) get(key);
-    }
-
-    public Long getLong(String key) {
-        return (Long) get(key);
-    }
-
-    public Double getDouble(String key) {
-        return (Double) get(key);
-    }
-
-    @SuppressWarnings("unchecked")
-    public List<String> getList(String key) {
-        return (List<String>) get(key);
-    }
-
-    public boolean getBoolean(String key) {
-        return (Boolean) get(key);
-    }
-
-    public String getString(String key) {
-        return (String) get(key);
-    }
-
-    public Class<?> getClass(String key) {
-        return (Class<?>) get(key);
-    }
-
-    public Set<String> unused() {
-        Set<String> keys = new HashSet<String>(originals.keySet());
-        keys.removeAll(used);
-        return keys;
-    }
-
-    public Map<String, Object> originals() {
-        Map<String, Object> copy = new HashMap<String, Object>();
-        copy.putAll(originals);
-        return copy;
-    }
-
-    private void logAll() {
-        StringBuilder b = new StringBuilder();
-        b.append(getClass().getSimpleName());
-        b.append(" values: ");
-        b.append(Utils.NL);
-        for (Map.Entry<String, Object> entry : this.values.entrySet()) {
-            b.append('\t');
-            b.append(entry.getKey());
-            b.append(" = ");
-            b.append(entry.getValue());
-            b.append(Utils.NL);
-        }
-        log.info(b.toString());
-    }
-
-    /**
-     * Log warnings for any unused configurations
-     */
-    public void logUnused() {
-        for (String key : unused())
-            log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.values.get(key));
-    }
-
-    /**
-     * Get a configured instance of the give class specified by the given configuration key. If the object implements
-     * Configurable configure it using the configuration.
-     * 
-     * @param key The configuration key for the class
-     * @param t The interface the class should implement
-     * @return A configured instance of the class
-     */
-    public <T> T getConfiguredInstance(String key, Class<T> t) {
-        Class<?> c = getClass(key);
-        if (c == null)
-            return null;
-        Object o = Utils.newInstance(c);
-        if (!t.isInstance(o))
-            throw new KafkaException(c.getName() + " is not an instance of " + t.getName());
-        if (o instanceof Configurable)
-            ((Configurable) o).configure(this.originals);
-        return t.cast(o);
-    }
-
-    public <T> List<T> getConfiguredInstances(String key, Class<T> t) {
-        List<String> klasses = getList(key);
-        List<T> objects = new ArrayList<T>();
-        for (String klass : klasses) {
-            Class<?> c;
-            try {
-                c = Class.forName(klass);
-            } catch (ClassNotFoundException e) {
-                throw new ConfigException(key, klass, "Class " + klass + " could not be found.");
-            }
-            if (c == null)
-                return null;
-            Object o = Utils.newInstance(c);
-            if (!t.isInstance(o))
-                throw new KafkaException(c.getName() + " is not an instance of " + t.getName());
-            if (o instanceof Configurable)
-                ((Configurable) o).configure(this.originals);
-            objects.add(t.cast(o));
-        }
-        return objects;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigDef.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigDef.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigDef.java
deleted file mode 100644
index 1bbe891..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigDef.java
+++ /dev/null
@@ -1,456 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.config;
-
-import org.apache.flink.kafka_backport.common.utils.Utils;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This class is used for specifying the set of expected configurations, their type, their defaults, their
- * documentation, and any special validation logic used for checking the correctness of the values the user provides.
- * <p/>
- * Usage of this class looks something like this:
- * <p/>
- * <pre>
- * ConfigDef defs = new ConfigDef();
- * defs.define(&quot;config_name&quot;, Type.STRING, &quot;default string value&quot;, &quot;This configuration is used for blah blah blah.&quot;);
- * defs.define(&quot;another_config_name&quot;, Type.INT, 42, Range.atLeast(0), &quot;More documentation on this config&quot;);
- *
- * Properties props = new Properties();
- * props.setProperty(&quot;config_name&quot;, &quot;some value&quot;);
- * Map&lt;String, Object&gt; configs = defs.parse(props);
- *
- * String someConfig = (String) configs.get(&quot;config_name&quot;); // will return &quot;some value&quot;
- * int anotherConfig = (Integer) configs.get(&quot;another_config_name&quot;); // will return default value of 42
- * </pre>
- * <p/>
- * This class can be used stand-alone or in combination with {@link AbstractConfig} which provides some additional
- * functionality for accessing configs.
- */
-public class ConfigDef {
-
-    private static final Object NO_DEFAULT_VALUE = new String("");
-
-    private final Map<String, ConfigKey> configKeys = new HashMap<String, ConfigKey>();
-
-    /**
-     * Returns unmodifiable set of properties names defined in this {@linkplain ConfigDef}
-     *
-     * @return new unmodifiable {@link Set} instance containing the keys
-     */
-    public Set<String> names() {
-        return Collections.unmodifiableSet(configKeys.keySet());
-    }
-
-    /**
-     * Define a new configuration
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param defaultValue  The default value to use if this config isn't present
-     * @param validator     A validator to use in checking the correctness of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @param required      Should the config fail if given property is not set and doesn't have default value specified
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation,
-                            boolean required) {
-        if (configKeys.containsKey(name))
-            throw new ConfigException("Configuration " + name + " is defined twice.");
-        Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type);
-        configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation, required));
-        return this;
-    }
-
-    /**
-     * Define a new required configuration
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param defaultValue  The default value to use if this config isn't present
-     * @param validator     A validator to use in checking the correctness of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation) {
-        return define(name, type, defaultValue, validator, importance, documentation, true);
-    }
-
-    /**
-     * Define a new configuration with no special validation logic
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param defaultValue  The default value to use if this config isn't present
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Object defaultValue, Importance importance, String documentation) {
-        return define(name, type, defaultValue, null, importance, documentation, true);
-    }
-
-    /**
-     * Define a required parameter with no default value
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param validator     A validator to use in checking the correctness of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Validator validator, Importance importance, String documentation) {
-        return define(name, type, NO_DEFAULT_VALUE, validator, importance, documentation, true);
-    }
-
-    /**
-     * Define a required parameter with no default value and no special validation logic
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Importance importance, String documentation) {
-        return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, true);
-    }
-
-    /**
-     * Define a required parameter with no default value and no special validation logic
-     *
-     * @param name          The name of the config parameter
-     * @param type          The type of the config
-     * @param importance    The importance of this config: is this something you will likely need to change.
-     * @param documentation The documentation string for the config
-     * @param required      Should the config fail if given property is not set and doesn't have default value specified
-     * @return This ConfigDef so you can chain calls
-     */
-    public ConfigDef define(String name, Type type, Importance importance, String documentation, boolean required) {
-        return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, required);
-    }
-
-
-    /**
-     * Parse and validate configs against this configuration definition. The input is a map of configs. It is expected
-     * that the keys of the map are strings, but the values can either be strings or they may already be of the
-     * appropriate type (int, string, etc). This will work equally well with either java.util.Properties instances or a
-     * programmatically constructed map.
-     *
-     * @param props The configs to parse and validate
-     * @return Parsed and validated configs. The key will be the config name and the value will be the value parsed into
-     * the appropriate type (int, string, etc)
-     */
-    public Map<String, Object> parse(Map<?, ?> props) {
-        /* parse all known keys */
-        Map<String, Object> values = new HashMap<String, Object>();
-        for (ConfigKey key : configKeys.values()) {
-            Object value;
-            // props map contains setting - assign ConfigKey value
-            if (props.containsKey(key.name))
-                value = parseType(key.name, props.get(key.name), key.type);
-                // props map doesn't contain setting, the key is required and no default value specified - it's an error
-            else if (key.defaultValue == NO_DEFAULT_VALUE && key.required)
-                throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value.");
-                // props map doesn't contain setting, no default value specified and the key is not required - assign it to null
-            else if (!key.hasDefault() && !key.required)
-                value = null;
-                // otherwise assign setting it's default value
-            else
-                value = key.defaultValue;
-            if (key.validator != null)
-                key.validator.ensureValid(key.name, value);
-            values.put(key.name, value);
-        }
-        return values;
-    }
-
-    /**
-     * Parse a value according to its expected type.
-     *
-     * @param name  The config name
-     * @param value The config value
-     * @param type  The expected type
-     * @return The parsed object
-     */
-    private Object parseType(String name, Object value, Type type) {
-        try {
-            String trimmed = null;
-            if (value instanceof String)
-                trimmed = ((String) value).trim();
-            switch (type) {
-                case BOOLEAN:
-                    if (value instanceof String) {
-                        if (trimmed.equalsIgnoreCase("true"))
-                            return true;
-                        else if (trimmed.equalsIgnoreCase("false"))
-                            return false;
-                        else
-                            throw new ConfigException(name, value, "Expected value to be either true or false");
-                    } else if (value instanceof Boolean)
-                        return value;
-                    else
-                        throw new ConfigException(name, value, "Expected value to be either true or false");
-                case STRING:
-                    if (value instanceof String)
-                        return trimmed;
-                    else
-                        throw new ConfigException(name, value, "Expected value to be a string, but it was a " + value.getClass().getName());
-                case INT:
-                    if (value instanceof Integer) {
-                        return (Integer) value;
-                    } else if (value instanceof String) {
-                        return Integer.parseInt(trimmed);
-                    } else {
-                        throw new ConfigException(name, value, "Expected value to be an number.");
-                    }
-                case SHORT:
-                    if (value instanceof Short) {
-                        return (Short) value;
-                    } else if (value instanceof String) {
-                        return Short.parseShort(trimmed);
-                    } else {
-                        throw new ConfigException(name, value, "Expected value to be an number.");
-                    }
-                case LONG:
-                    if (value instanceof Integer)
-                        return ((Integer) value).longValue();
-                    if (value instanceof Long)
-                        return (Long) value;
-                    else if (value instanceof String)
-                        return Long.parseLong(trimmed);
-                    else
-                        throw new ConfigException(name, value, "Expected value to be an number.");
-                case DOUBLE:
-                    if (value instanceof Number)
-                        return ((Number) value).doubleValue();
-                    else if (value instanceof String)
-                        return Double.parseDouble(trimmed);
-                    else
-                        throw new ConfigException(name, value, "Expected value to be an number.");
-                case LIST:
-                    if (value instanceof List)
-                        return (List<?>) value;
-                    else if (value instanceof String)
-                        if (trimmed.isEmpty())
-                            return Collections.emptyList();
-                        else
-                            return Arrays.asList(trimmed.split("\\s*,\\s*", -1));
-                    else
-                        throw new ConfigException(name, value, "Expected a comma separated list.");
-                case CLASS:
-                    if (value instanceof Class)
-                        return (Class<?>) value;
-                    else if (value instanceof String)
-                        return Class.forName(trimmed);
-                    else
-                        throw new ConfigException(name, value, "Expected a Class instance or class name.");
-                default:
-                    throw new IllegalStateException("Unknown type.");
-            }
-        } catch (NumberFormatException e) {
-            throw new ConfigException(name, value, "Not a number of type " + type);
-        } catch (ClassNotFoundException e) {
-            throw new ConfigException(name, value, "Class " + value + " could not be found.");
-        }
-    }
-
-    /**
-     * The config types
-     */
-    public enum Type {
-        BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS;
-    }
-
-    public enum Importance {
-        HIGH, MEDIUM, LOW
-    }
-
-    /**
-     * Validation logic the user may provide
-     */
-    public interface Validator {
-        public void ensureValid(String name, Object o);
-    }
-
-    /**
-     * Validation logic for numeric ranges
-     */
-    public static class Range implements Validator {
-        private final Number min;
-        private final Number max;
-
-        private Range(Number min, Number max) {
-            this.min = min;
-            this.max = max;
-        }
-
-        /**
-         * A numeric range that checks only the lower bound
-         *
-         * @param min The minimum acceptable value
-         */
-        public static Range atLeast(Number min) {
-            return new Range(min, null);
-        }
-
-        /**
-         * A numeric range that checks both the upper and lower bound
-         */
-        public static Range between(Number min, Number max) {
-            return new Range(min, max);
-        }
-
-        public void ensureValid(String name, Object o) {
-            Number n = (Number) o;
-            if (min != null && n.doubleValue() < min.doubleValue())
-                throw new ConfigException(name, o, "Value must be at least " + min);
-            if (max != null && n.doubleValue() > max.doubleValue())
-                throw new ConfigException(name, o, "Value must be no more than " + max);
-        }
-
-        public String toString() {
-            if (min == null)
-                return "[...," + max + "]";
-            else if (max == null)
-                return "[" + min + ",...]";
-            else
-                return "[" + min + ",...," + max + "]";
-        }
-    }
-
-    public static class ValidString implements Validator {
-        List<String> validStrings;
-
-        private ValidString(List<String> validStrings) {
-            this.validStrings = validStrings;
-        }
-
-        public static ValidString in(String... validStrings) {
-            return new ValidString(Arrays.asList(validStrings));
-        }
-
-        @Override
-        public void ensureValid(String name, Object o) {
-            String s = (String) o;
-            if (!validStrings.contains(s)) {
-                throw new ConfigException(name, o, "String must be one of: " + Utils.join(validStrings, ", "));
-            }
-
-        }
-
-        public String toString() {
-            return "[" + Utils.join(validStrings, ", ") + "]";
-        }
-    }
-
-    private static class ConfigKey {
-        public final String name;
-        public final Type type;
-        public final String documentation;
-        public final Object defaultValue;
-        public final Validator validator;
-        public final Importance importance;
-        public final boolean required;
-
-        public ConfigKey(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, boolean required) {
-            super();
-            this.name = name;
-            this.type = type;
-            this.defaultValue = defaultValue;
-            this.validator = validator;
-            this.importance = importance;
-            if (this.validator != null)
-                this.validator.ensureValid(name, defaultValue);
-            this.documentation = documentation;
-            this.required = required;
-        }
-
-        public boolean hasDefault() {
-            return this.defaultValue != NO_DEFAULT_VALUE;
-        }
-
-    }
-
-    public String toHtmlTable() {
-        // sort first required fields, then by importance, then name
-        List<ConfigKey> configs = new ArrayList<ConfigKey>(this.configKeys.values());
-        Collections.sort(configs, new Comparator<ConfigKey>() {
-            public int compare(ConfigDef.ConfigKey k1, ConfigDef.ConfigKey k2) {
-                // first take anything with no default value
-                if (!k1.hasDefault() && k2.hasDefault())
-                    return -1;
-                else if (!k2.hasDefault() && k1.hasDefault())
-                    return 1;
-
-                // then sort by importance
-                int cmp = k1.importance.compareTo(k2.importance);
-                if (cmp == 0)
-                    // then sort in alphabetical order
-                    return k1.name.compareTo(k2.name);
-                else
-                    return cmp;
-            }
-        });
-        StringBuilder b = new StringBuilder();
-        b.append("<table>\n");
-        b.append("<tr>\n");
-        b.append("<th>Name</th>\n");
-        b.append("<th>Type</th>\n");
-        b.append("<th>Default</th>\n");
-        b.append("<th>Importance</th>\n");
-        b.append("<th>Description</th>\n");
-        b.append("</tr>\n");
-        for (ConfigKey def : configs) {
-            b.append("<tr>\n");
-            b.append("<td>");
-            b.append(def.name);
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.type.toString().toLowerCase());
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.defaultValue == null ? "" : def.defaultValue);
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.importance.toString().toLowerCase());
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.documentation);
-            b.append("</td>");
-            b.append("</tr>\n");
-        }
-        b.append("</table>");
-        return b.toString();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigException.java
deleted file mode 100644
index 13b9410..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigException.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.kafka_backport.common.config;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Thrown if the user supplies an invalid configuration
- */
-public class ConfigException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public ConfigException(String message) {
-        super(message);
-    }
-
-    public ConfigException(String name, Object value) {
-        this(name, value, null);
-    }
-
-    public ConfigException(String name, Object value, String message) {
-        super("Invalid value " + value + " for configuration " + name + (message == null ? "" : ": " + message));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ApiException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ApiException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ApiException.java
deleted file mode 100644
index 1e6f7ec..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ApiException.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.kafka_backport.common.errors;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Any API exception that is part of the public protocol and should be a subclass of this class and be part of this
- * package.
- */
-public class ApiException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public ApiException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public ApiException(String message) {
-        super(message);
-    }
-
-    public ApiException(Throwable cause) {
-        super(cause);
-    }
-
-    public ApiException() {
-        super();
-    }
-
-    /* avoid the expensive and useless stack trace for api exceptions */
-    @Override
-    public Throwable fillInStackTrace() {
-        return this;
-    }
-
-}


[23/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/NetworkClient.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/NetworkClient.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/NetworkClient.java
new file mode 100644
index 0000000..90258bb
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/NetworkClient.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.kafka_backport.clients;
+
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.network.NetworkReceive;
+import org.apache.flink.kafka_backport.common.network.Send;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.requests.MetadataRequest;
+import org.apache.flink.kafka_backport.common.requests.MetadataResponse;
+import org.apache.flink.kafka_backport.common.requests.RequestHeader;
+import org.apache.flink.kafka_backport.common.requests.RequestSend;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+import org.apache.flink.kafka_backport.common.Cluster;
+import org.apache.flink.kafka_backport.common.network.Selectable;
+import org.apache.flink.kafka_backport.common.requests.ResponseHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A network client for asynchronous request/response network i/o. This is an internal class used to implement the
+ * user-facing producer and consumer clients.
+ * <p>
+ * This class is not thread-safe!
+ */
+public class NetworkClient implements KafkaClient {
+
+    private static final Logger log = LoggerFactory.getLogger(NetworkClient.class);
+
+    /* the selector used to perform network i/o */
+    private final Selectable selector;
+
+    /* the current cluster metadata */
+    private final Metadata metadata;
+
+    /* the state of each node's connection */
+    private final ClusterConnectionStates connectionStates;
+
+    /* the set of requests currently being sent or awaiting a response */
+    private final InFlightRequests inFlightRequests;
+
+    /* the socket send buffer size in bytes */
+    private final int socketSendBuffer;
+
+    /* the socket receive size buffer in bytes */
+    private final int socketReceiveBuffer;
+
+    /* the client id used to identify this client in requests to the server */
+    private final String clientId;
+
+    /* a random offset to use when choosing nodes to avoid having all nodes choose the same node */
+    private final int nodeIndexOffset;
+
+    /* the current correlation id to use when sending requests to servers */
+    private int correlation;
+
+    /* true iff there is a metadata request that has been sent and for which we have not yet received a response */
+    private boolean metadataFetchInProgress;
+
+    /* the last timestamp when no broker node is available to connect */
+    private long lastNoNodeAvailableMs;
+
+    public NetworkClient(Selectable selector,
+                         Metadata metadata,
+                         String clientId,
+                         int maxInFlightRequestsPerConnection,
+                         long reconnectBackoffMs,
+                         int socketSendBuffer,
+                         int socketReceiveBuffer) {
+        this.selector = selector;
+        this.metadata = metadata;
+        this.clientId = clientId;
+        this.inFlightRequests = new InFlightRequests(maxInFlightRequestsPerConnection);
+        this.connectionStates = new ClusterConnectionStates(reconnectBackoffMs);
+        this.socketSendBuffer = socketSendBuffer;
+        this.socketReceiveBuffer = socketReceiveBuffer;
+        this.correlation = 0;
+        this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE);
+        this.metadataFetchInProgress = false;
+        this.lastNoNodeAvailableMs = 0;
+    }
+
+    /**
+     * Begin connecting to the given node, return true if we are already connected and ready to send to that node.
+     * 
+     * @param node The node to check
+     * @param now The current timestamp
+     * @return True if we are ready to send to the given node
+     */
+    @Override
+    public boolean ready(Node node, long now) {
+        if (isReady(node, now))
+            return true;
+
+        if (connectionStates.canConnect(node.idString(), now))
+            // if we are interested in sending to a node and we don't have a connection to it, initiate one
+            initiateConnect(node, now);
+
+        return false;
+    }
+
+    /**
+     * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
+     * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
+     * connections.
+     * 
+     * @param node The node to check
+     * @param now The current timestamp
+     * @return The number of milliseconds to wait.
+     */
+    @Override
+    public long connectionDelay(Node node, long now) {
+        return connectionStates.connectionDelay(node.idString(), now);
+    }
+
+    /**
+     * Check if the connection of the node has failed, based on the connection state. Such connection failure are
+     * usually transient and can be resumed in the next {@link #ready(Node, long)} }
+     * call, but there are cases where transient failures needs to be caught and re-acted upon.
+     *
+     * @param node the node to check
+     * @return true iff the connection has failed and the node is disconnected
+     */
+    @Override
+    public boolean connectionFailed(Node node) {
+        return connectionStates.connectionState(node.idString()).equals(ConnectionState.DISCONNECTED);
+    }
+
+    /**
+     * Check if the node with the given id is ready to send more requests.
+     * 
+     * @param node The node
+     * @param now The current time in ms
+     * @return true if the node is ready
+     */
+    @Override
+    public boolean isReady(Node node, long now) {
+        String nodeId = node.idString();
+        if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0)
+            // if we need to update our metadata now declare all requests unready to make metadata requests first
+            // priority
+            return false;
+        else
+            // otherwise we are ready if we are connected and can send more requests
+            return isSendable(nodeId);
+    }
+
+    /**
+     * Are we connected and ready and able to send more requests to the given connection?
+     * 
+     * @param node The node
+     */
+    private boolean isSendable(String node) {
+        return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node);
+    }
+
+    /**
+     * Return the state of the connection to the given node
+     * 
+     * @param node The node to check
+     * @return The connection state
+     */
+    public ConnectionState connectionState(String node) {
+        return connectionStates.connectionState(node);
+    }
+
+    /**
+     * Queue up the given request for sending. Requests can only be sent out to ready nodes.
+     * 
+     * @param request The request
+     */
+    @Override
+    public void send(ClientRequest request) {
+        String nodeId = request.request().destination();
+        if (!isSendable(nodeId))
+            throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready.");
+
+        this.inFlightRequests.add(request);
+        selector.send(request.request());
+    }
+
+    /**
+     * Do actual reads and writes to sockets.
+     * 
+     * @param timeout The maximum amount of time to wait (in ms) for responses if there are none immediately
+     * @param now The current time in milliseconds
+     * @return The list of responses received
+     */
+    @Override
+    public List<ClientResponse> poll(long timeout, long now) {
+        // should we update our metadata?
+        long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now);
+        long timeToNextReconnectAttempt = Math.max(this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now, 0);
+        long waitForMetadataFetch = this.metadataFetchInProgress ? Integer.MAX_VALUE : 0;
+        // if there is no node available to connect, back off refreshing metadata
+        long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt),
+                waitForMetadataFetch);
+        if (metadataTimeout == 0)
+            maybeUpdateMetadata(now);
+        // do the I/O
+        try {
+            this.selector.poll(Math.min(timeout, metadataTimeout));
+        } catch (IOException e) {
+            log.error("Unexpected error during I/O in producer network thread", e);
+        }
+
+        // process completed actions
+        List<ClientResponse> responses = new ArrayList<ClientResponse>();
+        handleCompletedSends(responses, now);
+        handleCompletedReceives(responses, now);
+        handleDisconnections(responses, now);
+        handleConnections();
+
+        // invoke callbacks
+        for (ClientResponse response : responses) {
+            if (response.request().hasCallback()) {
+                try {
+                    response.request().callback().onComplete(response);
+                } catch (Exception e) {
+                    log.error("Uncaught error in request completion:", e);
+                }
+            }
+        }
+
+        return responses;
+    }
+
+    /**
+     * Await all the outstanding responses for requests on the given connection
+     * 
+     * @param node The node to block on
+     * @param now The current time in ms
+     * @return All the collected responses
+     */
+    @Override
+    public List<ClientResponse> completeAll(String node, long now) {
+        try {
+            this.selector.muteAll();
+            this.selector.unmute(node);
+            List<ClientResponse> responses = new ArrayList<ClientResponse>();
+            while (inFlightRequestCount(node) > 0)
+                responses.addAll(poll(Integer.MAX_VALUE, now));
+            return responses;
+        } finally {
+            this.selector.unmuteAll();
+        }
+    }
+
+    /**
+     * Wait for all outstanding requests to complete.
+     */
+    @Override
+    public List<ClientResponse> completeAll(long now) {
+        List<ClientResponse> responses = new ArrayList<ClientResponse>();
+        while (inFlightRequestCount() > 0)
+            responses.addAll(poll(Integer.MAX_VALUE, now));
+        return responses;
+    }
+
+    /**
+     * Get the number of in-flight requests
+     */
+    @Override
+    public int inFlightRequestCount() {
+        return this.inFlightRequests.inFlightRequestCount();
+    }
+
+    /**
+     * Get the number of in-flight requests for a given node
+     */
+    @Override
+    public int inFlightRequestCount(String node) {
+        return this.inFlightRequests.inFlightRequestCount(node);
+    }
+
+    /**
+     * Generate a request header for the given API key
+     * 
+     * @param key The api key
+     * @return A request header with the appropriate client id and correlation id
+     */
+    @Override
+    public RequestHeader nextRequestHeader(ApiKeys key) {
+        return new RequestHeader(key.id, clientId, correlation++);
+    }
+
+    /**
+     * Interrupt the client if it is blocked waiting on I/O.
+     */
+    @Override
+    public void wakeup() {
+        this.selector.wakeup();
+    }
+
+    /**
+     * Close the network client
+     */
+    @Override
+    public void close() {
+        this.selector.close();
+    }
+
+    /**
+     * Choose the node with the fewest outstanding requests which is at least eligible for connection. This method will
+     * prefer a node with an existing connection, but will potentially choose a node for which we don't yet have a
+     * connection if all existing connections are in use. This method will never choose a node for which there is no
+     * existing connection and from which we have disconnected within the reconnect backoff period.
+     * 
+     * @return The node with the fewest in-flight requests.
+     */
+    public Node leastLoadedNode(long now) {
+        List<Node> nodes = this.metadata.fetch().nodes();
+        int inflight = Integer.MAX_VALUE;
+        Node found = null;
+        for (int i = 0; i < nodes.size(); i++) {
+            int idx = Utils.abs((this.nodeIndexOffset + i) % nodes.size());
+            Node node = nodes.get(idx);
+            int currInflight = this.inFlightRequests.inFlightRequestCount(node.idString());
+            if (currInflight == 0 && this.connectionStates.isConnected(node.idString())) {
+                // if we find an established connection with no in-flight requests we can stop right away
+                return node;
+            } else if (!this.connectionStates.isBlackedOut(node.idString(), now) && currInflight < inflight) {
+                // otherwise if this is the best we have found so far, record that
+                inflight = currInflight;
+                found = node;
+            }
+        }
+        return found;
+    }
+
+    /**
+     * Handle any completed request send. In particular if no response is expected consider the request complete.
+     * 
+     * @param responses The list of responses to update
+     * @param now The current time
+     */
+    private void handleCompletedSends(List<ClientResponse> responses, long now) {
+        // if no response is expected then when the send is completed, return it
+        for (Send send : this.selector.completedSends()) {
+            ClientRequest request = this.inFlightRequests.lastSent(send.destination());
+            if (!request.expectResponse()) {
+                this.inFlightRequests.completeLastSent(send.destination());
+                responses.add(new ClientResponse(request, now, false, null));
+            }
+        }
+    }
+
+    /**
+     * Handle any completed receives and update the response list with the responses received.
+     * 
+     * @param responses The list of responses to update
+     * @param now The current time
+     */
+    private void handleCompletedReceives(List<ClientResponse> responses, long now) {
+        for (NetworkReceive receive : this.selector.completedReceives()) {
+            String source = receive.source();
+            ClientRequest req = inFlightRequests.completeNext(source);
+            ResponseHeader header = ResponseHeader.parse(receive.payload());
+            short apiKey = req.request().header().apiKey();
+            Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload());
+            correlate(req.request().header(), header);
+            if (apiKey == ApiKeys.METADATA.id) {
+                handleMetadataResponse(req.request().header(), body, now);
+            } else {
+                // need to add body/header to response here
+                responses.add(new ClientResponse(req, now, false, body));
+            }
+        }
+    }
+
+    private void handleMetadataResponse(RequestHeader header, Struct body, long now) {
+        this.metadataFetchInProgress = false;
+        MetadataResponse response = new MetadataResponse(body);
+        Cluster cluster = response.cluster();
+        // check if any topics metadata failed to get updated
+        if (response.errors().size() > 0) {
+            log.warn("Error while fetching metadata with correlation id {} : {}", header.correlationId(), response.errors());
+        }
+        // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being
+        // created which means we will get errors and no nodes until it exists
+        if (cluster.nodes().size() > 0) {
+            this.metadata.update(cluster, now);
+        } else {
+            log.trace("Ignoring empty metadata response with correlation id {}.", header.correlationId());
+            this.metadata.failedUpdate(now);
+        }
+    }
+
+    /**
+     * Handle any disconnected connections
+     * 
+     * @param responses The list of responses that completed with the disconnection
+     * @param now The current time
+     */
+    private void handleDisconnections(List<ClientResponse> responses, long now) {
+        for (String node : this.selector.disconnected()) {
+            connectionStates.disconnected(node);
+            log.debug("Node {} disconnected.", node);
+            for (ClientRequest request : this.inFlightRequests.clearAll(node)) {
+                log.trace("Cancelled request {} due to node {} being disconnected", request, node);
+                ApiKeys requestKey = ApiKeys.forId(request.request().header().apiKey());
+                if (requestKey == ApiKeys.METADATA)
+                    metadataFetchInProgress = false;
+                else
+                    responses.add(new ClientResponse(request, now, true, null));
+            }
+        }
+        // we got a disconnect so we should probably refresh our metadata and see if that broker is dead
+        if (this.selector.disconnected().size() > 0)
+            this.metadata.requestUpdate();
+    }
+
+    /**
+     * Record any newly completed connections
+     */
+    private void handleConnections() {
+        for (String node : this.selector.connected()) {
+            log.debug("Completed connection to node {}", node);
+            this.connectionStates.connected(node);
+        }
+    }
+
+    /**
+     * Validate that the response corresponds to the request we expect or else explode
+     */
+    private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) {
+        if (requestHeader.correlationId() != responseHeader.correlationId())
+            throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId()
+                    + ") does not match request (" + requestHeader.correlationId() + ")");
+    }
+
+    /**
+     * Create a metadata request for the given topics
+     */
+    private ClientRequest metadataRequest(long now, String node, Set<String> topics) {
+        MetadataRequest metadata = new MetadataRequest(new ArrayList<String>(topics));
+        RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct());
+        return new ClientRequest(now, true, send, null);
+    }
+
+    /**
+     * Add a metadata request to the list of sends if we can make one
+     */
+    private void maybeUpdateMetadata(long now) {
+        // Beware that the behavior of this method and the computation of timeouts for poll() are
+        // highly dependent on the behavior of leastLoadedNode.
+        Node node = this.leastLoadedNode(now);
+        if (node == null) {
+            log.debug("Give up sending metadata request since no node is available");
+            // mark the timestamp for no node available to connect
+            this.lastNoNodeAvailableMs = now;
+            return;
+        }
+        String nodeConnectionId = node.idString();
+
+
+        if (connectionStates.isConnected(nodeConnectionId) && inFlightRequests.canSendMore(nodeConnectionId)) {
+            Set<String> topics = metadata.topics();
+            this.metadataFetchInProgress = true;
+            ClientRequest metadataRequest = metadataRequest(now, nodeConnectionId, topics);
+            log.debug("Sending metadata request {} to node {}", metadataRequest, node.id());
+            this.selector.send(metadataRequest.request());
+            this.inFlightRequests.add(metadataRequest);
+        } else if (connectionStates.canConnect(nodeConnectionId, now)) {
+            // we don't have a connection to this node right now, make one
+            log.debug("Initialize connection to node {} for sending metadata request", node.id());
+            initiateConnect(node, now);
+            // If initiateConnect failed immediately, this node will be put into blackout and we
+            // should allow immediately retrying in case there is another candidate node. If it
+            // is still connecting, the worst case is that we end up setting a longer timeout
+            // on the next round and then wait for the response.
+        } else { // connected, but can't send more OR connecting
+            // In either case, we just need to wait for a network event to let us know the selected
+            // connection might be usable again.
+            this.lastNoNodeAvailableMs = now;
+        }
+    }
+
+    /**
+     * Initiate a connection to the given node
+     */
+    private void initiateConnect(Node node, long now) {
+        String nodeConnectionId = node.idString();
+        try {
+            log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port());
+            this.connectionStates.connecting(nodeConnectionId, now);
+            selector.connect(nodeConnectionId,
+                             new InetSocketAddress(node.host(), node.port()),
+                             this.socketSendBuffer,
+                             this.socketReceiveBuffer);
+        } catch (IOException e) {
+            /* attempt failed, we'll try again after the backoff */
+            connectionStates.disconnected(nodeConnectionId);
+            /* maybe the problem is our metadata, update it */
+            metadata.requestUpdate();
+            log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/RequestCompletionHandler.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/RequestCompletionHandler.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/RequestCompletionHandler.java
new file mode 100644
index 0000000..a275e96
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/RequestCompletionHandler.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.kafka_backport.clients;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A callback interface for attaching an action to be executed when a request is complete and the corresponding response
+ * has been received. This handler will also be invoked if there is a disconnection while handling the request.
+ */
+public interface RequestCompletionHandler {
+
+    public void onComplete(ClientResponse response);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/CommitType.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/CommitType.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/CommitType.java
new file mode 100644
index 0000000..d261bd6
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/CommitType.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public enum CommitType {
+    SYNC, ASYNC
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/Consumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/Consumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/Consumer.java
new file mode 100644
index 0000000..12b48ec
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/Consumer.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.kafka_backport.clients.consumer;
+
+import org.apache.flink.kafka_backport.common.Metric;
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.apache.flink.kafka_backport.common.PartitionInfo;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+
+import java.io.Closeable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * @see KafkaConsumer
+ * @see MockConsumer
+ */
+public interface Consumer<K, V> extends Closeable {
+    
+    /**
+     * @see KafkaConsumer#subscriptions()
+     */
+    public Set<TopicPartition> subscriptions();
+
+    /**
+     * @see KafkaConsumer#subscribe(String...)
+     */
+    public void subscribe(String... topics);
+
+    /**
+     * @see KafkaConsumer#subscribe(TopicPartition...)
+     */
+    public void subscribe(TopicPartition... partitions);
+
+    /**
+     * @see KafkaConsumer#unsubscribe(String...)
+     */
+    public void unsubscribe(String... topics);
+
+    /**
+     * @see KafkaConsumer#unsubscribe(TopicPartition...)
+     */
+    public void unsubscribe(TopicPartition... partitions);
+
+    /**
+     * @see KafkaConsumer#poll(long)
+     */
+    public ConsumerRecords<K, V> poll(long timeout);
+
+    /**
+     * @see KafkaConsumer#commit(CommitType)
+     */
+    public void commit(CommitType commitType);
+
+    /**
+     * @see KafkaConsumer#commit(CommitType, ConsumerCommitCallback)
+     */
+    public void commit(CommitType commitType, ConsumerCommitCallback callback);
+
+    /**
+     * @see KafkaConsumer#commit(Map, CommitType)
+     */
+    public void commit(Map<TopicPartition, Long> offsets, CommitType commitType);
+
+    /**
+     * @see KafkaConsumer#commit(Map, CommitType, ConsumerCommitCallback)
+     */
+    public void commit(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback);
+
+    /**
+     * @see KafkaConsumer#seek(TopicPartition, long)
+     */
+    public void seek(TopicPartition partition, long offset);
+
+    /**
+     * @see KafkaConsumer#seekToBeginning(TopicPartition...)
+     */
+    public void seekToBeginning(TopicPartition... partitions);
+
+    /**
+     * @see KafkaConsumer#seekToEnd(TopicPartition...)
+     */
+    public void seekToEnd(TopicPartition... partitions);
+
+    /**
+     * @see KafkaConsumer#position(TopicPartition)
+     */
+    public long position(TopicPartition partition);
+
+    /**
+     * @see KafkaConsumer#committed(TopicPartition)
+     */
+    public long committed(TopicPartition partition);
+
+    /**
+     * @see KafkaConsumer#metrics()
+     */
+    public Map<MetricName, ? extends Metric> metrics();
+
+    /**
+     * @see KafkaConsumer#partitionsFor(String)
+     */
+    public List<PartitionInfo> partitionsFor(String topic);
+
+    /**
+     * @see KafkaConsumer#close()
+     */
+    public void close();
+
+    /**
+     * @see KafkaConsumer#wakeup()
+     */
+    public void wakeup();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerCommitCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerCommitCallback.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerCommitCallback.java
new file mode 100644
index 0000000..4d90bfc
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerCommitCallback.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer;
+
+import org.apache.flink.kafka_backport.common.TopicPartition;
+
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A callback interface that the user can implement to trigger custom actions when a commit request completes. The callback
+ * may be executed in any thread calling {@link Consumer#poll(long) poll()}.
+ */
+public interface ConsumerCommitCallback {
+
+    /**
+     * A callback method the user can implement to provide asynchronous handling of commit request completion.
+     * This method will be called when the commit request sent to the server has been acknowledged.
+     *
+     * @param offsets A map of the offsets that this callback applies to
+     * @param exception The exception thrown during processing of the request, or null if the commit completed successfully
+     */
+    void onComplete(Map<TopicPartition, Long> offsets, Exception exception);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerConfig.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerConfig.java
new file mode 100644
index 0000000..c1f79bf
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerConfig.java
@@ -0,0 +1,334 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer;
+
+import org.apache.flink.kafka_backport.clients.CommonClientConfigs;
+import org.apache.flink.kafka_backport.clients.consumer.internals.NoOpConsumerRebalanceCallback;
+import org.apache.flink.kafka_backport.common.config.AbstractConfig;
+import org.apache.flink.kafka_backport.common.config.ConfigDef;
+import org.apache.flink.kafka_backport.common.config.ConfigDef.Importance;
+import org.apache.flink.kafka_backport.common.config.ConfigDef.Type;
+import org.apache.flink.kafka_backport.common.serialization.Deserializer;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.flink.kafka_backport.common.config.ConfigDef.Range.atLeast;
+import static org.apache.flink.kafka_backport.common.config.ConfigDef.ValidString.in;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The consumer configuration keys
+ */
+public class ConsumerConfig extends AbstractConfig {
+    private static final ConfigDef CONFIG;
+
+    /*
+     * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS
+     * THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
+     */
+
+    /**
+     * <code>group.id</code>
+     */
+    public static final String GROUP_ID_CONFIG = "group.id";
+    private static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using <code>subscribe(topic)</code> or the Kafka-based offset management strategy.";
+
+    /**
+     * <code>session.timeout.ms</code>
+     */
+    public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms";
+    private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities.";
+
+    /**
+     * <code>bootstrap.servers</code>
+     */
+    public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
+
+    /**
+     * <code>enable.auto.commit</code>
+     */
+    public static final String ENABLE_AUTO_COMMIT_CONFIG = "enable.auto.commit";
+    private static final String ENABLE_AUTO_COMMIT_DOC = "If true the consumer's offset will be periodically committed in the background.";
+
+    /**
+     * <code>auto.commit.interval.ms</code>
+     */
+    public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms";
+    private static final String AUTO_COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if <code>enable.auto.commit</code> is set to <code>true</code>.";
+
+    /**
+     * <code>partition.assignment.strategy</code>
+     */
+    public static final String PARTITION_ASSIGNMENT_STRATEGY_CONFIG = "partition.assignment.strategy";
+    private static final String PARTITION_ASSIGNMENT_STRATEGY_DOC = "The friendly name of the partition assignment strategy that the server will use to distribute partition ownership amongst consumer instances when group management is used";
+
+    /**
+     * <code>auto.offset.reset</code>
+     */
+    public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset";
+    private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted): <ul><li>smallest: automatically reset the offset to the smallest offset<li>largest: automatically reset the offset to the largest offset</li><li>none: throw exception to the consumer if no previous offset is found for the consumer's group</li><li>anything else: throw exception to the consumer.</li></ul>";
+
+    /**
+     * <code>fetch.min.bytes</code>
+     */
+    public static final String FETCH_MIN_BYTES_CONFIG = "fetch.min.bytes";
+    private static final String FETCH_MIN_BYTES_DOC = "The minimum amount of data the server should return for a fetch request. If insufficient data is available the request will wait for that much data to accumulate before answering the request. The default setting of 1 byte means that fetch requests are answered as soon as a single byte of data is available or the fetch request times out waiting for data to arrive. Setting this to something greater than 1 will cause the server to wait for larger amounts of data to accumulate which can improve server throughput a bit at the cost of some additional latency.";
+
+    /**
+     * <code>fetch.max.wait.ms</code>
+     */
+    public static final String FETCH_MAX_WAIT_MS_CONFIG = "fetch.max.wait.ms";
+    private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch.min.bytes.";
+
+    /** <code>metadata.max.age.ms</code> */
+    public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG;
+
+    /**
+     * <code>max.partition.fetch.bytes</code>
+     */
+    public static final String MAX_PARTITION_FETCH_BYTES_CONFIG = "max.partition.fetch.bytes";
+    private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server will return. The maximum total memory used for a request will be <code>#partitions * max.partition.fetch.bytes</code>. This size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large message on a certain partition.";
+
+    /** <code>send.buffer.bytes</code> */
+    public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG;
+
+    /** <code>receive.buffer.bytes</code> */
+    public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG;
+
+    /**
+     * <code>client.id</code>
+     */
+    public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
+
+    /**
+     * <code>reconnect.backoff.ms</code>
+     */
+    public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG;
+
+    /**
+     * <code>retry.backoff.ms</code>
+     */
+    public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG;
+
+    /**
+     * <code>metrics.sample.window.ms</code>
+     */
+    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
+
+    /**
+     * <code>metrics.num.samples</code>
+     */
+    public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG;
+
+    /**
+     * <code>metric.reporters</code>
+     */
+    public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG;
+
+    /**
+     * <code>rebalance.callback.class</code>
+     */
+    public static final String CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG = "rebalance.callback.class";
+    private static final String CONSUMER_REBALANCE_CALLBACK_CLASS_DOC = "A user-provided callback to execute when partition assignments change.";
+
+    /**
+     * <code>check.crcs</code>
+     */
+    public static final String CHECK_CRCS_CONFIG = "check.crcs";
+    private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance.";
+    
+    /** <code>key.deserializer</code> */
+    public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer";
+    private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the <code>Deserializer</code> interface.";
+
+    /** <code>value.deserializer</code> */
+    public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer";
+    private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the <code>Deserializer</code> interface.";
+
+    /** <code>connections.max.idle.ms</code> */
+    public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;
+
+
+    static {
+        CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
+                                        Type.LIST,
+                                        Importance.HIGH,
+                                        CommonClientConfigs.BOOSTRAP_SERVERS_DOC)
+                                .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC)
+                                .define(SESSION_TIMEOUT_MS_CONFIG,
+                                        Type.INT,
+                                        30000,
+                                        Importance.HIGH,
+                                        SESSION_TIMEOUT_MS_DOC)
+                                .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
+                                        Type.STRING,
+                                        "range",
+                                        in("range", "roundrobin"),
+                                        Importance.MEDIUM,
+                                        PARTITION_ASSIGNMENT_STRATEGY_DOC)
+                                .define(METADATA_MAX_AGE_CONFIG,
+                                        Type.LONG,
+                                        5 * 60 * 1000,
+                                        atLeast(0),
+                                        Importance.LOW,
+                                        CommonClientConfigs.METADATA_MAX_AGE_DOC)
+                                .define(ENABLE_AUTO_COMMIT_CONFIG,
+                                        Type.BOOLEAN,
+                                        true,
+                                        Importance.MEDIUM,
+                                        ENABLE_AUTO_COMMIT_DOC)
+                                .define(AUTO_COMMIT_INTERVAL_MS_CONFIG,
+                                        Type.LONG,
+                                        5000,
+                                        atLeast(0),
+                                        Importance.LOW,
+                                        AUTO_COMMIT_INTERVAL_MS_DOC)
+                                .define(CLIENT_ID_CONFIG,
+                                        Type.STRING,
+                                        "",
+                                        Importance.LOW,
+                                        CommonClientConfigs.CLIENT_ID_DOC)
+                                .define(MAX_PARTITION_FETCH_BYTES_CONFIG,
+                                        Type.INT,
+                                        1 * 1024 * 1024,
+                                        atLeast(0),
+                                        Importance.HIGH,
+                                        MAX_PARTITION_FETCH_BYTES_DOC)
+                                .define(SEND_BUFFER_CONFIG,
+                                        Type.INT,
+                                        128 * 1024,
+                                        atLeast(0),
+                                        Importance.MEDIUM,
+                                        CommonClientConfigs.SEND_BUFFER_DOC)
+                                .define(RECEIVE_BUFFER_CONFIG,
+                                        Type.INT,
+                                        32 * 1024,
+                                        atLeast(0),
+                                        Importance.MEDIUM,
+                                        CommonClientConfigs.RECEIVE_BUFFER_DOC)
+                                .define(FETCH_MIN_BYTES_CONFIG,
+                                        Type.INT,
+                                        1024,
+                                        atLeast(0),
+                                        Importance.HIGH,
+                                        FETCH_MIN_BYTES_DOC)
+                                .define(FETCH_MAX_WAIT_MS_CONFIG,
+                                        Type.INT,
+                                        500,
+                                        atLeast(0),
+                                        Importance.LOW,
+                                        FETCH_MAX_WAIT_MS_DOC)
+                                .define(RECONNECT_BACKOFF_MS_CONFIG,
+                                        Type.LONG,
+                                        50L,
+                                        atLeast(0L),
+                                        Importance.LOW,
+                                        CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC)
+                                .define(RETRY_BACKOFF_MS_CONFIG,
+                                        Type.LONG,
+                                        100L,
+                                        atLeast(0L),
+                                        Importance.LOW,
+                                        CommonClientConfigs.RETRY_BACKOFF_MS_DOC)
+                                .define(AUTO_OFFSET_RESET_CONFIG,
+                                        Type.STRING,
+                                        "latest",
+                                        in("latest", "earliest", "none"),
+                                        Importance.MEDIUM,
+                                        AUTO_OFFSET_RESET_DOC)
+                                .define(CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
+                                        Type.CLASS,
+                                        NoOpConsumerRebalanceCallback.class,
+                                        Importance.LOW,
+                                        CONSUMER_REBALANCE_CALLBACK_CLASS_DOC)
+                                .define(CHECK_CRCS_CONFIG,
+                                        Type.BOOLEAN,
+                                        true,
+                                        Importance.LOW,
+                                        CHECK_CRCS_DOC)                                
+                                .define(METRICS_SAMPLE_WINDOW_MS_CONFIG,
+                                        Type.LONG,
+                                        30000,
+                                        atLeast(0),
+                                        Importance.LOW,
+                                        CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC)
+                                .define(METRICS_NUM_SAMPLES_CONFIG,
+                                        Type.INT,
+                                        2,
+                                        atLeast(1),
+                                        Importance.LOW,
+                                        CommonClientConfigs.METRICS_NUM_SAMPLES_DOC)
+                                .define(METRIC_REPORTER_CLASSES_CONFIG,
+                                        Type.LIST,
+                                        "",
+                                        Importance.LOW,
+                                        CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC)
+                                .define(KEY_DESERIALIZER_CLASS_CONFIG,
+                                        Type.CLASS,
+                                        Importance.HIGH,
+                                        KEY_DESERIALIZER_CLASS_DOC)
+                                .define(VALUE_DESERIALIZER_CLASS_CONFIG,
+                                        Type.CLASS,
+                                        Importance.HIGH,
+                                        VALUE_DESERIALIZER_CLASS_DOC)
+                                /* default is set to be a bit lower than the server default (10 min), to avoid both client and server closing connection at same time */
+                                .define(CONNECTIONS_MAX_IDLE_MS_CONFIG,
+                                        Type.LONG,
+                                        9 * 60 * 1000,
+                                        Importance.MEDIUM,
+                                        CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC);
+    }
+
+    public static Map<String, Object> addDeserializerToConfig(Map<String, Object> configs,
+                                                              Deserializer<?> keyDeserializer,
+                                                              Deserializer<?> valueDeserializer) {
+        Map<String, Object> newConfigs = new HashMap<String, Object>();
+        newConfigs.putAll(configs);
+        if (keyDeserializer != null)
+            newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass());
+        if (keyDeserializer != null)
+            newConfigs.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass());
+        return newConfigs;
+    }
+
+    public static Properties addDeserializerToConfig(Properties properties,
+                                                     Deserializer<?> keyDeserializer,
+                                                     Deserializer<?> valueDeserializer) {
+        Properties newProperties = new Properties();
+        newProperties.putAll(properties);
+        if (keyDeserializer != null)
+            newProperties.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName());
+        if (keyDeserializer != null)
+            newProperties.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName());
+        return newProperties;
+    }
+
+    public ConsumerConfig(Map<?, ?> props) {
+        super(CONFIG, props);
+    }
+
+    public static void main(String[] args) {
+        System.out.println(CONFIG.toHtmlTable());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRebalanceCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRebalanceCallback.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRebalanceCallback.java
new file mode 100644
index 0000000..8f2cd75
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRebalanceCallback.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.kafka_backport.clients.consumer;
+
+import org.apache.flink.kafka_backport.common.TopicPartition;
+
+import java.util.Collection;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A callback interface that the user can implement to trigger custom actions when the set of partitions assigned to the
+ * consumer changes.
+ * <p>
+ * This is applicable when the consumer is having Kafka auto-manage group membership, if the consumer's directly subscribe to partitions
+ * those partitions will never be reassigned and this callback is not applicable.
+ * <p>
+ * When Kafka is managing the group membership, a partition re-assignment will be triggered any time the members of the group changes or the subscription
+ * of the members changes. This can occur when processes die, new process instances are added or old instances come back to life after failure.
+ * <p>
+ * There are many uses for this functionality. One common use is saving offsets in a custom store. By saving offsets in
+ * the {@link #onPartitionsRevoked(Consumer, Collection)} call we can ensure that any time partition assignment changes
+ * the offset gets saved.
+ * <p>
+ * Another use is flushing out any kind of cache of intermediate results the consumer may be keeping. For example,
+ * consider a case where the consumer is subscribed to a topic containing user page views, and the goal is to count the
+ * number of page views per users for each five minute window. Let's say the topic is partitioned by the user id so that
+ * all events for a particular user will go to a single consumer instance. The consumer can keep in memory a running
+ * tally of actions per user and only flush these out to a remote data store when it's cache gets to big. However if a
+ * partition is reassigned it may want to automatically trigger a flush of this cache, before the new owner takes over
+ * consumption.
+ * <p>
+ * This callback will execute in the user thread as part of the {@link Consumer#poll(long) poll(long)} call whenever partition assignment changes.
+ * <p>
+ * It is guaranteed that all consumer processes will invoke {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} prior to 
+ * any process invoking {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned}. So if offsets or other state is saved in the 
+ * {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} call it is guaranteed to be saved by the time the process taking over that
+ * partition has their {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned} callback called to load the state.
+ * <p>
+ * Here is pseudo-code for a callback implementation for saving offsets:
+ * <pre>
+ * {@code
+ *   public class SaveOffsetsOnRebalance implements ConsumerRebalanceCallback {
+ *       public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
+ *           // read the offsets from an external store using some custom code not described here
+ *           for(TopicPartition partition: partitions)
+ *              consumer.position(partition, readOffsetFromExternalStore(partition));
+ *       }      
+ *       public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
+ *           // save the offsets in an external store using some custom code not described here
+ *           for(TopicPartition partition: partitions)
+ *              saveOffsetInExternalStore(consumer.position(partition));
+ *       }
+ *   }
+ * }
+ * </pre>
+ */
+public interface ConsumerRebalanceCallback {
+
+    /**
+     * A callback method the user can implement to provide handling of customized offsets on completion of a successful
+     * partition re-assignement. This method will be called after an offset re-assignement completes and before the
+     * consumer starts fetching data.
+     * <p>
+     * It is guaranteed that all the processes in a consumer group will execute their
+     * {@link #onPartitionsRevoked(Consumer, Collection)} callback before any instance executes its
+     * {@link #onPartitionsAssigned(Consumer, Collection)} callback.
+     *
+     * @param consumer Reference to the consumer for convenience
+     * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously
+     *            assigned to the consumer)
+     */
+    public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
+
+    /**
+     * A callback method the user can implement to provide handling of offset commits to a customized store on the start
+     * of a rebalance operation. This method will be called before a rebalance operation starts and after the consumer
+     * stops fetching data. It is recommended that offsets should be committed in this callback to either Kafka or a
+     * custom offset store to prevent duplicate data
+     * <p>
+     * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}
+     *
+     * @param consumer  Reference to the consumer for convenience
+     * @param partitions The list of partitions that were assigned to the consumer on the last rebalance
+     */
+    public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecord.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecord.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecord.java
new file mode 100644
index 0000000..44096ee
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecord.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the
+ * record is being received and an offset that points to the record in a Kafka partition.
+ */
+public final class ConsumerRecord<K, V> {
+    private final String topic;
+    private final int partition;
+    private final long offset;
+    private final K key;
+    private final V value;
+
+    /**
+     * Create a record with no key
+     * 
+     * @param topic The topic this record is received from
+     * @param partition The partition of the topic this record is received from
+     * @param offset The offset of this record in the corresponding Kafka partition
+     * @param value The record contents
+     */
+    public ConsumerRecord(String topic, int partition, long offset, K key, V value) {
+        if (topic == null)
+            throw new IllegalArgumentException("Topic cannot be null");
+        this.topic = topic;
+        this.partition = partition;
+        this.offset = offset;
+        this.key = key;
+        this.value = value;
+    }
+
+    /**
+     * The topic this record is received from
+     */
+    public String topic() {
+        return this.topic;
+    }
+
+    /**
+     * The partition from which this record is received
+     */
+    public int partition() {
+        return this.partition;
+    }
+
+    /**
+     * The key (or null if no key is specified)
+     */
+    public K key() {
+        return key;
+    }
+
+    /**
+     * The value
+     */
+    public V value() {
+        return value;
+    }
+
+    /**
+     * The position of this record in the corresponding Kafka partition.
+     */
+    public long offset() {
+        return offset;
+    }
+
+    @Override
+    public String toString() {
+        return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset()
+                + ", key = " + key + ", value = " + value + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecords.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecords.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecords.java
new file mode 100644
index 0000000..a9cfc84
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecords.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.kafka_backport.clients.consumer;
+
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.utils.AbstractIterator;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A container that holds the list {@link ConsumerRecord} per partition for a
+ * particular topic. There is one for every topic returned by a
+ * {@link Consumer#poll(long)} operation.
+ */
+public class ConsumerRecords<K, V> implements Iterable<ConsumerRecord<K, V>> {
+    public static final ConsumerRecords<Object, Object> EMPTY =
+            new ConsumerRecords<Object, Object>(Collections.EMPTY_MAP);
+
+    private final Map<TopicPartition, List<ConsumerRecord<K, V>>> records;
+
+    public ConsumerRecords(Map<TopicPartition, List<ConsumerRecord<K, V>>> records) {
+        this.records = records;
+    }
+
+    /**
+     * Get just the records for the given partition
+     * 
+     * @param partition The partition to get records for
+     */
+    public Iterable<ConsumerRecord<K, V>> records(TopicPartition partition) {
+        List<ConsumerRecord<K, V>> recs = this.records.get(partition);
+        if (recs == null)
+            return Collections.emptyList();
+        else
+            return recs;
+    }
+
+    /**
+     * Get just the records for the given topic
+     */
+    public Iterable<ConsumerRecord<K, V>> records(String topic) {
+        if (topic == null)
+            throw new IllegalArgumentException("Topic must be non-null.");
+        List<List<ConsumerRecord<K, V>>> recs = new ArrayList<List<ConsumerRecord<K, V>>>();
+        for (Map.Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : records.entrySet()) {
+            if (entry.getKey().topic().equals(topic))
+                recs.add(entry.getValue());
+        }
+        return new ConcatenatedIterable<K, V>(recs);
+    }
+
+    @Override
+    public Iterator<ConsumerRecord<K, V>> iterator() {
+        return new ConcatenatedIterable<K, V>(records.values()).iterator();
+    }
+    
+    /**
+     * The number of records for all topics
+     */
+    public int count() {
+        int count = 0;
+        for (List<ConsumerRecord<K, V>> recs: this.records.values())
+            count += recs.size();
+        return count;
+    }
+
+    private static class ConcatenatedIterable<K, V> implements Iterable<ConsumerRecord<K, V>> {
+
+        private final Iterable<? extends Iterable<ConsumerRecord<K, V>>> iterables;
+
+        public ConcatenatedIterable(Iterable<? extends Iterable<ConsumerRecord<K, V>>> iterables) {
+            this.iterables = iterables;
+        }
+
+        @Override
+        public Iterator<ConsumerRecord<K, V>> iterator() {
+            return new AbstractIterator<ConsumerRecord<K, V>>() {
+                Iterator<? extends Iterable<ConsumerRecord<K, V>>> iters = iterables.iterator();
+                Iterator<ConsumerRecord<K, V>> current;
+
+                public ConsumerRecord<K, V> makeNext() {
+                    if (current == null || !current.hasNext()) {
+                        if (iters.hasNext())
+                            current = iters.next().iterator();
+                        else
+                            return allDone();
+                    }
+                    return current.next();
+                }
+            };
+        }
+    }
+
+    public boolean isEmpty() {
+        return records.isEmpty();
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <K, V> ConsumerRecords<K, V> empty() {
+        return (ConsumerRecords<K, V>) EMPTY;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerWakeupException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerWakeupException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerWakeupException.java
new file mode 100644
index 0000000..7185c87
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerWakeupException.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.kafka_backport.clients.consumer;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ConsumerWakeupException extends KafkaException {
+    private static final long serialVersionUID = 1L;
+
+}


[40/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchRequest.java
deleted file mode 100644
index f797ebe..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchRequest.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class FetchRequest extends AbstractRequest {
-    
-    public static final int CONSUMER_REPLICA_ID = -1;
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id);
-    private static final String REPLICA_ID_KEY_NAME = "replica_id";
-    private static final String MAX_WAIT_KEY_NAME = "max_wait_time";
-    private static final String MIN_BYTES_KEY_NAME = "min_bytes";
-    private static final String TOPICS_KEY_NAME = "topics";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset";
-    private static final String MAX_BYTES_KEY_NAME = "max_bytes";
-
-    private final int replicaId;
-    private final int maxWait;
-    private final int minBytes;
-    private final Map<TopicPartition, PartitionData> fetchData;
-
-    public static final class PartitionData {
-        public final long offset;
-        public final int maxBytes;
-
-        public PartitionData(long offset, int maxBytes) {
-            this.offset = offset;
-            this.maxBytes = maxBytes;
-        }
-    }
-
-    /**
-     * Create a non-replica fetch request
-     */
-    public FetchRequest(int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
-        this(CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData);
-    }
-
-    /**
-     * Create a replica fetch request
-     */
-    public FetchRequest(int replicaId, int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(fetchData);
-
-        struct.set(REPLICA_ID_KEY_NAME, replicaId);
-        struct.set(MAX_WAIT_KEY_NAME, maxWait);
-        struct.set(MIN_BYTES_KEY_NAME, minBytes);
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry : topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset);
-                partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
-        this.replicaId = replicaId;
-        this.maxWait = maxWait;
-        this.minBytes = minBytes;
-        this.fetchData = fetchData;
-    }
-
-    public FetchRequest(Struct struct) {
-        super(struct);
-        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
-        maxWait = struct.getInt(MAX_WAIT_KEY_NAME);
-        minBytes = struct.getInt(MIN_BYTES_KEY_NAME);
-        fetchData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                long offset = partitionResponse.getLong(FETCH_OFFSET_KEY_NAME);
-                int maxBytes = partitionResponse.getInt(MAX_BYTES_KEY_NAME);
-                PartitionData partitionData = new PartitionData(offset, maxBytes);
-                fetchData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<TopicPartition, FetchResponse.PartitionData> responseData = new HashMap<TopicPartition, FetchResponse.PartitionData>();
-
-        for (Map.Entry<TopicPartition, PartitionData> entry: fetchData.entrySet()) {
-            FetchResponse.PartitionData partitionResponse = new FetchResponse.PartitionData(Errors.forException(e).code(),
-                    FetchResponse.INVALID_HIGHWATERMARK,
-                    FetchResponse.EMPTY_RECORD_SET);
-            responseData.put(entry.getKey(), partitionResponse);
-        }
-
-        switch (versionId) {
-            case 0:
-                return new FetchResponse(responseData);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.FETCH.id)));
-        }
-    }
-
-    public int replicaId() {
-        return replicaId;
-    }
-
-    public int maxWait() {
-        return maxWait;
-    }
-
-    public int minBytes() {
-        return minBytes;
-    }
-
-    public Map<TopicPartition, PartitionData> fetchData() {
-        return fetchData;
-    }
-
-    public static FetchRequest parse(ByteBuffer buffer, int versionId) {
-        return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer));
-    }
-
-    public static FetchRequest parse(ByteBuffer buffer) {
-        return new FetchRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchResponse.java
deleted file mode 100644
index 158833e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchResponse.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class FetchResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     *  OFFSET_OUT_OF_RANGE (1)
-     *  UNKNOWN_TOPIC_OR_PARTITION (3)
-     *  NOT_LEADER_FOR_PARTITION (6)
-     *  REPLICA_NOT_AVAILABLE (9)
-     *  UNKNOWN (-1)
-     */
-
-    private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark";
-    private static final String RECORD_SET_KEY_NAME = "record_set";
-
-    public static final long INVALID_HIGHWATERMARK = -1L;
-    public static final ByteBuffer EMPTY_RECORD_SET = ByteBuffer.allocate(0);
-
-    private final Map<TopicPartition, PartitionData> responseData;
-
-    public static final class PartitionData {
-        public final short errorCode;
-        public final long highWatermark;
-        public final ByteBuffer recordSet;
-
-        public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) {
-            this.errorCode = errorCode;
-            this.highWatermark = highWatermark;
-            this.recordSet = recordSet;
-        }
-    }
-
-    public FetchResponse(Map<TopicPartition, PartitionData> responseData) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
-                partitionData.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark);
-                partitionData.set(RECORD_SET_KEY_NAME, fetchPartitionData.recordSet);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
-        this.responseData = responseData;
-    }
-
-    public FetchResponse(Struct struct) {
-        super(struct);
-        responseData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
-                long highWatermark = partitionResponse.getLong(HIGH_WATERMARK_KEY_NAME);
-                ByteBuffer recordSet = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
-                PartitionData partitionData = new PartitionData(errorCode, highWatermark, recordSet);
-                responseData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    public Map<TopicPartition, PartitionData> responseData() {
-        return responseData;
-    }
-
-    public static FetchResponse parse(ByteBuffer buffer) {
-        return new FetchResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatRequest.java
deleted file mode 100644
index c8abb67..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatRequest.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class HeartbeatRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-    private static final String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id";
-    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
-
-    private final String groupId;
-    private final int groupGenerationId;
-    private final String consumerId;
-
-    public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        this.groupId = groupId;
-        this.groupGenerationId = groupGenerationId;
-        this.consumerId = consumerId;
-    }
-
-    public HeartbeatRequest(Struct struct) {
-        super(struct);
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-        groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME);
-        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        switch (versionId) {
-            case 0:
-                return new HeartbeatResponse(Errors.forException(e).code());
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.HEARTBEAT.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public int groupGenerationId() {
-        return groupGenerationId;
-    }
-
-    public String consumerId() {
-        return consumerId;
-    }
-
-    public static HeartbeatRequest parse(ByteBuffer buffer, int versionId) {
-        return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer));
-    }
-
-    public static HeartbeatRequest parse(ByteBuffer buffer) {
-        return new HeartbeatRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatResponse.java
deleted file mode 100644
index 4bf6669..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatResponse.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class HeartbeatResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id);
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR_FOR_CONSUMER (16)
-     * ILLEGAL_GENERATION (22)
-     * UNKNOWN_CONSUMER_ID (25)
-     */
-
-    private final short errorCode;
-    public HeartbeatResponse(short errorCode) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(ERROR_CODE_KEY_NAME, errorCode);
-        this.errorCode = errorCode;
-    }
-
-    public HeartbeatResponse(Struct struct) {
-        super(struct);
-        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
-    }
-
-    public short errorCode() {
-        return errorCode;
-    }
-
-    public static HeartbeatResponse parse(ByteBuffer buffer) {
-        return new HeartbeatResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupRequest.java
deleted file mode 100644
index f098d18..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupRequest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class JoinGroupRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-    private static final String SESSION_TIMEOUT_KEY_NAME = "session_timeout";
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
-    private static final String STRATEGY_KEY_NAME = "partition_assignment_strategy";
-
-    public static final String UNKNOWN_CONSUMER_ID = "";
-
-    private final String groupId;
-    private final int sessionTimeout;
-    private final List<String> topics;
-    private final String consumerId;
-    private final String strategy;
-
-    public JoinGroupRequest(String groupId, int sessionTimeout, List<String> topics, String consumerId, String strategy) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout);
-        struct.set(TOPICS_KEY_NAME, topics.toArray());
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        struct.set(STRATEGY_KEY_NAME, strategy);
-        this.groupId = groupId;
-        this.sessionTimeout = sessionTimeout;
-        this.topics = topics;
-        this.consumerId = consumerId;
-        this.strategy = strategy;
-    }
-
-    public JoinGroupRequest(Struct struct) {
-        super(struct);
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-        sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME);
-        Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME);
-        topics = new ArrayList<String>();
-        for (Object topic: topicsArray)
-            topics.add((String) topic);
-        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
-        strategy = struct.getString(STRATEGY_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        switch (versionId) {
-            case 0:
-                return new JoinGroupResponse(
-                        Errors.forException(e).code(),
-                        JoinGroupResponse.UNKNOWN_GENERATION_ID,
-                        JoinGroupResponse.UNKNOWN_CONSUMER_ID,
-                        Collections.<TopicPartition>emptyList());
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public int sessionTimeout() {
-        return sessionTimeout;
-    }
-
-    public List<String> topics() {
-        return topics;
-    }
-
-    public String consumerId() {
-        return consumerId;
-    }
-
-    public String strategy() {
-        return strategy;
-    }
-
-    public static JoinGroupRequest parse(ByteBuffer buffer, int versionId) {
-        return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, versionId, buffer));
-    }
-
-    public static JoinGroupRequest parse(ByteBuffer buffer) {
-        return new JoinGroupRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupResponse.java
deleted file mode 100644
index 7d9b647..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupResponse.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class JoinGroupResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id);
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR_FOR_CONSUMER (16)
-     * INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY (23)
-     * UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY (24)
-     * UNKNOWN_CONSUMER_ID (25)
-     * INVALID_SESSION_TIMEOUT (26)
-     */
-
-    private static final String GENERATION_ID_KEY_NAME = "group_generation_id";
-    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
-    private static final String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions";
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    public static final int UNKNOWN_GENERATION_ID = -1;
-    public static final String UNKNOWN_CONSUMER_ID = "";
-
-    private final short errorCode;
-    private final int generationId;
-    private final String consumerId;
-    private final List<TopicPartition> assignedPartitions;
-
-    public JoinGroupResponse(short errorCode, int generationId, String consumerId, List<TopicPartition> assignedPartitions) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        Map<String, List<Integer>> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions);
-
-        struct.set(ERROR_CODE_KEY_NAME, errorCode);
-        struct.set(GENERATION_ID_KEY_NAME, generationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, List<Integer>> entries: partitionsByTopic.entrySet()) {
-            Struct topicData = struct.instance(ASSIGNED_PARTITIONS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
-            topicData.set(PARTITIONS_KEY_NAME, entries.getValue().toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(ASSIGNED_PARTITIONS_KEY_NAME, topicArray.toArray());
-
-        this.errorCode = errorCode;
-        this.generationId = generationId;
-        this.consumerId = consumerId;
-        this.assignedPartitions = assignedPartitions;
-    }
-
-    public JoinGroupResponse(Struct struct) {
-        super(struct);
-        assignedPartitions = new ArrayList<TopicPartition>();
-        for (Object topicDataObj : struct.getArray(ASSIGNED_PARTITIONS_KEY_NAME)) {
-            Struct topicData = (Struct) topicDataObj;
-            String topic = topicData.getString(TOPIC_KEY_NAME);
-            for (Object partitionObj : topicData.getArray(PARTITIONS_KEY_NAME))
-                assignedPartitions.add(new TopicPartition(topic, (Integer) partitionObj));
-        }
-        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
-        generationId = struct.getInt(GENERATION_ID_KEY_NAME);
-        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
-    }
-
-    public short errorCode() {
-        return errorCode;
-    }
-
-    public int generationId() {
-        return generationId;
-    }
-
-    public String consumerId() {
-        return consumerId;
-    }
-
-    public List<TopicPartition> assignedPartitions() {
-        return assignedPartitions;
-    }
-
-    public static JoinGroupResponse parse(ByteBuffer buffer) {
-        return new JoinGroupResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetRequest.java
deleted file mode 100644
index 069e06d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetRequest.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ListOffsetRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id);
-    private static final String REPLICA_ID_KEY_NAME = "replica_id";
-    private static final String TOPICS_KEY_NAME = "topics";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String TIMESTAMP_KEY_NAME = "timestamp";
-    private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets";
-
-    private final int replicaId;
-    private final Map<TopicPartition, PartitionData> offsetData;
-
-    public static final class PartitionData {
-        public final long timestamp;
-        public final int maxNumOffsets;
-
-        public PartitionData(long timestamp, int maxNumOffsets) {
-            this.timestamp = timestamp;
-            this.maxNumOffsets = maxNumOffsets;
-        }
-    }
-    
-    public ListOffsetRequest(Map<TopicPartition, PartitionData> offsetData) {
-        this(-1, offsetData);
-    }
-
-    public ListOffsetRequest(int replicaId, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
-
-        struct.set(REPLICA_ID_KEY_NAME, replicaId);
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData offsetPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp);
-                partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
-        this.replicaId = replicaId;
-        this.offsetData = offsetData;
-    }
-
-    public ListOffsetRequest(Struct struct) {
-        super(struct);
-        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
-        offsetData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
-                int maxNumOffsets = partitionResponse.getInt(MAX_NUM_OFFSETS_KEY_NAME);
-                PartitionData partitionData = new PartitionData(timestamp, maxNumOffsets);
-                offsetData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<TopicPartition, ListOffsetResponse.PartitionData> responseData = new HashMap<TopicPartition, ListOffsetResponse.PartitionData>();
-
-        for (Map.Entry<TopicPartition, PartitionData> entry: offsetData.entrySet()) {
-            ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(Errors.forException(e).code(), new ArrayList<Long>());
-            responseData.put(entry.getKey(), partitionResponse);
-        }
-
-        switch (versionId) {
-            case 0:
-                return new ListOffsetResponse(responseData);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id)));
-        }
-    }
-
-    public int replicaId() {
-        return replicaId;
-    }
-
-    public Map<TopicPartition, PartitionData> offsetData() {
-        return offsetData;
-    }
-
-    public static ListOffsetRequest parse(ByteBuffer buffer, int versionId) {
-        return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer));
-    }
-
-    public static ListOffsetRequest parse(ByteBuffer buffer) {
-        return new ListOffsetRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetResponse.java
deleted file mode 100644
index b831f61..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetResponse.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ListOffsetResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     *  UNKNOWN_TOPIC_OR_PARTITION (3)
-     *  NOT_LEADER_FOR_PARTITION (6)
-     *  UNKNOWN (-1)
-     */
-
-    private static final String OFFSETS_KEY_NAME = "offsets";
-
-    private final Map<TopicPartition, PartitionData> responseData;
-
-    public static final class PartitionData {
-        public final short errorCode;
-        public final List<Long> offsets;
-
-        public PartitionData(short errorCode, List<Long> offsets) {
-            this.errorCode = errorCode;
-            this.offsets = offsets;
-        }
-    }
-
-    public ListOffsetResponse(Map<TopicPartition, PartitionData> responseData) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData offsetPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.errorCode);
-                partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray());
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
-        this.responseData = responseData;
-    }
-
-    public ListOffsetResponse(Struct struct) {
-        super(struct);
-        responseData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
-                Object[] offsets = partitionResponse.getArray(OFFSETS_KEY_NAME);
-                List<Long> offsetsList = new ArrayList<Long>();
-                for (Object offset: offsets)
-                    offsetsList.add((Long) offset);
-                PartitionData partitionData = new PartitionData(errorCode, offsetsList);
-                responseData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    public Map<TopicPartition, PartitionData> responseData() {
-        return responseData;
-    }
-
-    public static ListOffsetResponse parse(ByteBuffer buffer) {
-        return new ListOffsetResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataRequest.java
deleted file mode 100644
index 2820fcd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataRequest.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.Node;
-import org.apache.flink.kafka_backport.common.PartitionInfo;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.Cluster;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class MetadataRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id);
-    private static final String TOPICS_KEY_NAME = "topics";
-
-    private final List<String> topics;
-
-    public MetadataRequest(List<String> topics) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(TOPICS_KEY_NAME, topics.toArray());
-        this.topics = topics;
-    }
-
-    public MetadataRequest(Struct struct) {
-        super(struct);
-        Object[] topicArray = struct.getArray(TOPICS_KEY_NAME);
-        topics = new ArrayList<String>();
-        for (Object topicObj: topicArray) {
-            topics.add((String) topicObj);
-        }
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<String, Errors> topicErrors = new HashMap<String, Errors>();
-        for (String topic : topics) {
-            topicErrors.put(topic, Errors.forException(e));
-        }
-
-        Cluster cluster = new Cluster(new ArrayList<Node>(), new ArrayList<PartitionInfo>());
-        switch (versionId) {
-            case 0:
-                return new MetadataResponse(cluster, topicErrors);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id)));
-        }
-    }
-
-    public List<String> topics() {
-        return topics;
-    }
-
-    public static MetadataRequest parse(ByteBuffer buffer, int versionId) {
-        return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer));
-    }
-
-    public static MetadataRequest parse(ByteBuffer buffer) {
-        return new MetadataRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataResponse.java
deleted file mode 100644
index 83d7290..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataResponse.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.Node;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.Cluster;
-import org.apache.flink.kafka_backport.common.PartitionInfo;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class MetadataResponse extends AbstractRequestResponse {
-
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id);
-    private static final String BROKERS_KEY_NAME = "brokers";
-    private static final String TOPIC_METATDATA_KEY_NAME = "topic_metadata";
-
-    // broker level field names
-    private static final String NODE_ID_KEY_NAME = "node_id";
-    private static final String HOST_KEY_NAME = "host";
-    private static final String PORT_KEY_NAME = "port";
-
-    // topic level field names
-    private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code";
-
-    /**
-     * Possible error code:
-     *
-     * TODO
-     */
-
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata";
-
-    // partition level field names
-    private static final String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code";
-
-    /**
-     * Possible error code:
-     *
-     * TODO
-     */
-
-    private static final String PARTITION_KEY_NAME = "partition_id";
-    private static final String LEADER_KEY_NAME = "leader";
-    private static final String REPLICAS_KEY_NAME = "replicas";
-    private static final String ISR_KEY_NAME = "isr";
-
-    private final Cluster cluster;
-    private final Map<String, Errors> errors;
-
-    /**
-     * Constructor for MetadataResponse where there are errors for some of the topics,
-     * error data take precedence over cluster information for particular topic
-     */
-    public MetadataResponse(Cluster cluster, Map<String, Errors> errors) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        List<Struct> brokerArray = new ArrayList<Struct>();
-        for (Node node : cluster.nodes()) {
-            Struct broker = struct.instance(BROKERS_KEY_NAME);
-            broker.set(NODE_ID_KEY_NAME, node.id());
-            broker.set(HOST_KEY_NAME, node.host());
-            broker.set(PORT_KEY_NAME, node.port());
-            brokerArray.add(broker);
-        }
-        struct.set(BROKERS_KEY_NAME, brokerArray.toArray());
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (String topic : cluster.topics()) {
-            Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME);
-
-            topicData.set(TOPIC_KEY_NAME, topic);
-            if (errors.containsKey(topic)) {
-                topicData.set(TOPIC_ERROR_CODE_KEY_NAME, errors.get(topic).code());
-            } else {
-                topicData.set(TOPIC_ERROR_CODE_KEY_NAME, Errors.NONE.code());
-                List<Struct> partitionArray = new ArrayList<Struct>();
-                for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) {
-                    Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME);
-                    partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, Errors.NONE.code());
-                    partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition());
-                    partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id());
-                    ArrayList<Integer> replicas = new ArrayList<Integer>();
-                    for (Node node : fetchPartitionData.replicas())
-                        replicas.add(node.id());
-                    partitionData.set(REPLICAS_KEY_NAME, replicas.toArray());
-                    ArrayList<Integer> isr = new ArrayList<Integer>();
-                    for (Node node : fetchPartitionData.inSyncReplicas())
-                        isr.add(node.id());
-                    partitionData.set(ISR_KEY_NAME, isr.toArray());
-                    partitionArray.add(partitionData);
-                }
-                topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray());
-            }
-
-            topicArray.add(topicData);
-        }
-        struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray());
-
-        this.cluster = cluster;
-        this.errors = new HashMap<String, Errors>();
-    }
-
-    public MetadataResponse(Struct struct) {
-        super(struct);
-        Map<String, Errors> errors = new HashMap<String, Errors>();
-        Map<Integer, Node> brokers = new HashMap<Integer, Node>();
-        Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME);
-        for (int i = 0; i < brokerStructs.length; i++) {
-            Struct broker = (Struct) brokerStructs[i];
-            int nodeId = broker.getInt(NODE_ID_KEY_NAME);
-            String host = broker.getString(HOST_KEY_NAME);
-            int port = broker.getInt(PORT_KEY_NAME);
-            brokers.put(nodeId, new Node(nodeId, host, port));
-        }
-        List<PartitionInfo> partitions = new ArrayList<PartitionInfo>();
-        Object[] topicInfos = (Object[]) struct.get(TOPIC_METATDATA_KEY_NAME);
-        for (int i = 0; i < topicInfos.length; i++) {
-            Struct topicInfo = (Struct) topicInfos[i];
-            short topicError = topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME);
-            String topic = topicInfo.getString(TOPIC_KEY_NAME);
-            if (topicError == Errors.NONE.code()) {
-                Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME);
-                for (int j = 0; j < partitionInfos.length; j++) {
-                    Struct partitionInfo = (Struct) partitionInfos[j];
-                    int partition = partitionInfo.getInt(PARTITION_KEY_NAME);
-                    int leader = partitionInfo.getInt(LEADER_KEY_NAME);
-                    Node leaderNode = leader == -1 ? null : brokers.get(leader);
-                    Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME);
-                    Node[] replicaNodes = new Node[replicas.length];
-                    for (int k = 0; k < replicas.length; k++)
-                        replicaNodes[k] = brokers.get(replicas[k]);
-                    Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME);
-                    Node[] isrNodes = new Node[isr.length];
-                    for (int k = 0; k < isr.length; k++)
-                        isrNodes[k] = brokers.get(isr[k]);
-                    partitions.add(new PartitionInfo(topic, partition, leaderNode, replicaNodes, isrNodes));
-                }
-            } else {
-                errors.put(topic, Errors.forCode(topicError));
-            }
-        }
-        this.errors = errors;
-        this.cluster = new Cluster(brokers.values(), partitions);
-    }
-
-    public Map<String, Errors> errors() {
-        return this.errors;
-    }
-
-    public Cluster cluster() {
-        return this.cluster;
-    }
-
-    public static MetadataResponse parse(ByteBuffer buffer) {
-        return new MetadataResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitRequest.java
deleted file mode 100644
index b33d2c1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitRequest.java
+++ /dev/null
@@ -1,275 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This wrapper supports both v0 and v1 of OffsetCommitRequest.
- */
-public class OffsetCommitRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-    private static final String GENERATION_ID_KEY_NAME = "group_generation_id";
-    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String RETENTION_TIME_KEY_NAME = "retention_time";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
-    private static final String METADATA_KEY_NAME = "metadata";
-
-    @Deprecated
-    private static final String TIMESTAMP_KEY_NAME = "timestamp";         // for v0, v1
-
-    // default values for the current version
-    public static final int DEFAULT_GENERATION_ID = -1;
-    public static final String DEFAULT_CONSUMER_ID = "";
-    public static final long DEFAULT_RETENTION_TIME = -1L;
-
-    // default values for old versions,
-    // will be removed after these versions are deprecated
-    @Deprecated
-    public static final long DEFAULT_TIMESTAMP = -1L;            // for V0, V1
-
-    private final String groupId;
-    private final String consumerId;
-    private final int generationId;
-    private final long retentionTime;
-    private final Map<TopicPartition, PartitionData> offsetData;
-
-    public static final class PartitionData {
-        @Deprecated
-        public final long timestamp;                // for V1
-
-        public final long offset;
-        public final String metadata;
-
-        @Deprecated
-        public PartitionData(long offset, long timestamp, String metadata) {
-            this.offset = offset;
-            this.timestamp = timestamp;
-            this.metadata = metadata;
-        }
-
-        public PartitionData(long offset, String metadata) {
-            this(offset, DEFAULT_TIMESTAMP, metadata);
-        }
-    }
-
-    /**
-     * Constructor for version 0.
-     * @param groupId
-     * @param offsetData
-     */
-    @Deprecated
-    public OffsetCommitRequest(String groupId, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0)));
-
-        initCommonFields(groupId, offsetData);
-        this.groupId = groupId;
-        this.generationId = DEFAULT_GENERATION_ID;
-        this.consumerId = DEFAULT_CONSUMER_ID;
-        this.retentionTime = DEFAULT_RETENTION_TIME;
-        this.offsetData = offsetData;
-    }
-
-    /**
-     * Constructor for version 1.
-     * @param groupId
-     * @param generationId
-     * @param consumerId
-     * @param offsetData
-     */
-    @Deprecated
-    public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1)));
-
-        initCommonFields(groupId, offsetData);
-        struct.set(GENERATION_ID_KEY_NAME, generationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        this.groupId = groupId;
-        this.generationId = generationId;
-        this.consumerId = consumerId;
-        this.retentionTime = DEFAULT_RETENTION_TIME;
-        this.offsetData = offsetData;
-    }
-
-    /**
-     * Constructor for version 2.
-     * @param groupId
-     * @param generationId
-     * @param consumerId
-     * @param retentionTime
-     * @param offsetData
-     */
-    public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        initCommonFields(groupId, offsetData);
-        struct.set(GENERATION_ID_KEY_NAME, generationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        struct.set(RETENTION_TIME_KEY_NAME, retentionTime);
-        this.groupId = groupId;
-        this.generationId = generationId;
-        this.consumerId = consumerId;
-        this.retentionTime = retentionTime;
-        this.offsetData = offsetData;
-    }
-
-    private void initCommonFields(String groupId, Map<TopicPartition, PartitionData> offsetData) {
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
-
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        List<Struct> topicArray = new ArrayList<Struct>();
-
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
-                // Only for v1
-                if (partitionData.hasField(TIMESTAMP_KEY_NAME))
-                    partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp);
-                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
-    }
-
-    public OffsetCommitRequest(Struct struct) {
-        super(struct);
-
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-        // This field only exists in v1.
-        if (struct.hasField(GENERATION_ID_KEY_NAME))
-            generationId = struct.getInt(GENERATION_ID_KEY_NAME);
-        else
-            generationId = DEFAULT_GENERATION_ID;
-
-        // This field only exists in v1.
-        if (struct.hasField(CONSUMER_ID_KEY_NAME))
-            consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
-        else
-            consumerId = DEFAULT_CONSUMER_ID;
-
-        // This field only exists in v2
-        if (struct.hasField(RETENTION_TIME_KEY_NAME))
-            retentionTime = struct.getLong(RETENTION_TIME_KEY_NAME);
-        else
-            retentionTime = DEFAULT_RETENTION_TIME;
-
-        offsetData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicDataObj : struct.getArray(TOPICS_KEY_NAME)) {
-            Struct topicData = (Struct) topicDataObj;
-            String topic = topicData.getString(TOPIC_KEY_NAME);
-            for (Object partitionDataObj : topicData.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionDataStruct = (Struct) partitionDataObj;
-                int partition = partitionDataStruct.getInt(PARTITION_KEY_NAME);
-                long offset = partitionDataStruct.getLong(COMMIT_OFFSET_KEY_NAME);
-                String metadata = partitionDataStruct.getString(METADATA_KEY_NAME);
-                PartitionData partitionOffset;
-                // This field only exists in v1
-                if (partitionDataStruct.hasField(TIMESTAMP_KEY_NAME)) {
-                    long timestamp = partitionDataStruct.getLong(TIMESTAMP_KEY_NAME);
-                    partitionOffset = new PartitionData(offset, timestamp, metadata);
-                } else {
-                    partitionOffset = new PartitionData(offset, metadata);
-                }
-                offsetData.put(new TopicPartition(topic, partition), partitionOffset);
-            }
-        }
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<TopicPartition, Short> responseData = new HashMap<TopicPartition, Short>();
-        for (Map.Entry<TopicPartition, PartitionData> entry: offsetData.entrySet()) {
-            responseData.put(entry.getKey(), Errors.forException(e).code());
-        }
-
-        switch (versionId) {
-            // OffsetCommitResponseV0 == OffsetCommitResponseV1 == OffsetCommitResponseV2
-            case 0:
-            case 1:
-            case 2:
-                return new OffsetCommitResponse(responseData);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_COMMIT.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public int generationId() {
-        return generationId;
-    }
-
-    public String consumerId() {
-        return consumerId;
-    }
-
-    public long retentionTime() {
-        return retentionTime;
-    }
-
-    public Map<TopicPartition, PartitionData> offsetData() {
-        return offsetData;
-    }
-
-    public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) {
-        Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId);
-        return new OffsetCommitRequest((Struct) schema.read(buffer));
-    }
-
-    public static OffsetCommitRequest parse(ByteBuffer buffer) {
-        return new OffsetCommitRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitResponse.java
deleted file mode 100644
index 5f14b63..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitResponse.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class OffsetCommitResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level fields
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    // partition level fields
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     * OFFSET_METADATA_TOO_LARGE (12)
-     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR_FOR_CONSUMER (16)
-     * ILLEGAL_GENERATION (22)
-     * UNKNOWN_CONSUMER_ID (25)
-     * COMMITTING_PARTITIONS_NOT_ASSIGNED (27)
-     * INVALID_COMMIT_OFFSET_SIZE (28)
-     */
-
-    private final Map<TopicPartition, Short> responseData;
-
-    public OffsetCommitResponse(Map<TopicPartition, Short> responseData) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        Map<String, Map<Integer, Short>> topicsData = CollectionUtils.groupDataByTopic(responseData);
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, Short>> entries: topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, Short> partitionEntry : entries.getValue().entrySet()) {
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(ERROR_CODE_KEY_NAME, partitionEntry.getValue());
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
-        this.responseData = responseData;
-    }
-
-    public OffsetCommitResponse(Struct struct) {
-        super(struct);
-        responseData = new HashMap<TopicPartition, Short>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
-                responseData.put(new TopicPartition(topic, partition), errorCode);
-            }
-        }
-    }
-
-    public Map<TopicPartition, Short> responseData() {
-        return responseData;
-    }
-
-    public static OffsetCommitResponse parse(ByteBuffer buffer) {
-        return new OffsetCommitResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}


[45/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ConsumerCoordinatorNotAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ConsumerCoordinatorNotAvailableException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ConsumerCoordinatorNotAvailableException.java
deleted file mode 100644
index 1add30e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ConsumerCoordinatorNotAvailableException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has
- * not yet been created.
- */
-public class ConsumerCoordinatorNotAvailableException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public ConsumerCoordinatorNotAvailableException() {
-        super();
-    }
-
-    public ConsumerCoordinatorNotAvailableException(String message) {
-        super(message);
-    }
-
-    public ConsumerCoordinatorNotAvailableException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public ConsumerCoordinatorNotAvailableException(Throwable cause) {
-        super(cause);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/CorruptRecordException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/CorruptRecordException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/CorruptRecordException.java
deleted file mode 100644
index d5771df..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/CorruptRecordException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This exception indicates a record has failed it's internal CRC check, this generally indicates network or disk
- * corruption.
- */
-public class CorruptRecordException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public CorruptRecordException() {
-        super("This message has failed it's CRC checksum or is otherwise corrupt.");
-    }
-
-    public CorruptRecordException(String message) {
-        super(message);
-    }
-
-    public CorruptRecordException(Throwable cause) {
-        super(cause);
-    }
-
-    public CorruptRecordException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/DisconnectException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/DisconnectException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/DisconnectException.java
deleted file mode 100644
index 1c048d3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/DisconnectException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Server disconnected before a request could be completed.
- */
-public class DisconnectException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public DisconnectException() {
-        super();
-    }
-
-    public DisconnectException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public DisconnectException(String message) {
-        super(message);
-    }
-
-    public DisconnectException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/IllegalGenerationException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/IllegalGenerationException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/IllegalGenerationException.java
deleted file mode 100644
index ae46b5f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/IllegalGenerationException.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class IllegalGenerationException extends RetriableException {
-    private static final long serialVersionUID = 1L;
-
-    public IllegalGenerationException() {
-        super();
-    }
-
-    public IllegalGenerationException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public IllegalGenerationException(String message) {
-        super(message);
-    }
-
-    public IllegalGenerationException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InterruptException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InterruptException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InterruptException.java
deleted file mode 100644
index f7949f4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InterruptException.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.kafka_backport.common.errors;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * An unchecked wrapper for InterruptedException
- */
-public class InterruptException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-    
-    public InterruptException(InterruptedException cause) {
-        super(cause);
-        Thread.currentThread().interrupt();
-    }
-    
-    public InterruptException(String message, InterruptedException cause) {
-        super(message, cause);
-        Thread.currentThread().interrupt();
-    }
-
-    public InterruptException(String message) {
-        super(message, new InterruptedException());
-        Thread.currentThread().interrupt();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidMetadataException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidMetadataException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidMetadataException.java
deleted file mode 100644
index 710a391..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidMetadataException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * An exception that may indicate the client's metadata is out of date
- */
-public abstract class InvalidMetadataException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public InvalidMetadataException() {
-        super();
-    }
-
-    public InvalidMetadataException(String message) {
-        super(message);
-    }
-
-    public InvalidMetadataException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public InvalidMetadataException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidRequiredAcksException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidRequiredAcksException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidRequiredAcksException.java
deleted file mode 100644
index a3cd167..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidRequiredAcksException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class InvalidRequiredAcksException extends ApiException {
-    private static final long serialVersionUID = 1L;
-
-    public InvalidRequiredAcksException(String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidTopicException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidTopicException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidTopicException.java
deleted file mode 100644
index e0c5a41..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidTopicException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The client has attempted to perform an operation on an invalid topic.
- */
-public class InvalidTopicException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public InvalidTopicException() {
-        super();
-    }
-
-    public InvalidTopicException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public InvalidTopicException(String message) {
-        super(message);
-    }
-
-    public InvalidTopicException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/LeaderNotAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/LeaderNotAvailableException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/LeaderNotAvailableException.java
deleted file mode 100644
index 701a3c6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/LeaderNotAvailableException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * There is no currently available leader for the given partition (either because a leadership election is in progress
- * or because all replicas are down).
- */
-public class LeaderNotAvailableException extends InvalidMetadataException {
-
-    private static final long serialVersionUID = 1L;
-
-    public LeaderNotAvailableException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NetworkException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NetworkException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NetworkException.java
deleted file mode 100644
index ceca78f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NetworkException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A misc. network-related IOException occurred when making a request. This could be because the client's metadata is
- * out of date and it is making a request to a node that is now dead.
- */
-public class NetworkException extends InvalidMetadataException {
-
-    private static final long serialVersionUID = 1L;
-
-    public NetworkException() {
-        super();
-    }
-
-    public NetworkException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public NetworkException(String message) {
-        super(message);
-    }
-
-    public NetworkException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotCoordinatorForConsumerException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotCoordinatorForConsumerException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotCoordinatorForConsumerException.java
deleted file mode 100644
index 3aea94b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotCoordinatorForConsumerException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is
- * not a coordinator for.
- */
-public class NotCoordinatorForConsumerException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public NotCoordinatorForConsumerException() {
-        super();
-    }
-
-    public NotCoordinatorForConsumerException(String message) {
-        super(message);
-    }
-
-    public NotCoordinatorForConsumerException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public NotCoordinatorForConsumerException(Throwable cause) {
-        super(cause);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasAfterAppendException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasAfterAppendException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasAfterAppendException.java
deleted file mode 100644
index c2f8203..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasAfterAppendException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Number of insync replicas for the partition is lower than min.insync.replicas This exception is raised when the low
- * ISR size is discovered *after* the message was already appended to the log. Producer retries will cause duplicates.
- */
-public class NotEnoughReplicasAfterAppendException extends RetriableException {
-    private static final long serialVersionUID = 1L;
-
-    public NotEnoughReplicasAfterAppendException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasException.java
deleted file mode 100644
index 93eb850..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Number of insync replicas for the partition is lower than min.insync.replicas
- */
-public class NotEnoughReplicasException extends RetriableException {
-    private static final long serialVersionUID = 1L;
-
-    public NotEnoughReplicasException() {
-        super();
-    }
-
-    public NotEnoughReplicasException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public NotEnoughReplicasException(String message) {
-        super(message);
-    }
-
-    public NotEnoughReplicasException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotLeaderForPartitionException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotLeaderForPartitionException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotLeaderForPartitionException.java
deleted file mode 100644
index a7dafb4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotLeaderForPartitionException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This server is not the leader for the given partition
- */
-public class NotLeaderForPartitionException extends InvalidMetadataException {
-
-    private static final long serialVersionUID = 1L;
-
-    public NotLeaderForPartitionException() {
-        super();
-    }
-
-    public NotLeaderForPartitionException(String message) {
-        super(message);
-    }
-
-    public NotLeaderForPartitionException(Throwable cause) {
-        super(cause);
-    }
-
-    public NotLeaderForPartitionException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetLoadInProgressException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetLoadInProgressException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetLoadInProgressException.java
deleted file mode 100644
index f6a17b9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetLoadInProgressException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change
- * for that offsets topic partition).
- */
-public class OffsetLoadInProgressException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public OffsetLoadInProgressException() {
-        super();
-    }
-
-    public OffsetLoadInProgressException(String message) {
-        super(message);
-    }
-
-    public OffsetLoadInProgressException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public OffsetLoadInProgressException(Throwable cause) {
-        super(cause);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetMetadataTooLarge.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetMetadataTooLarge.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetMetadataTooLarge.java
deleted file mode 100644
index 6a57b5d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetMetadataTooLarge.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The client has tried to save its offset with associated metadata larger than the maximum size allowed by the server.
- */
-public class OffsetMetadataTooLarge extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public OffsetMetadataTooLarge() {
-    }
-
-    public OffsetMetadataTooLarge(String message) {
-        super(message);
-    }
-
-    public OffsetMetadataTooLarge(Throwable cause) {
-        super(cause);
-    }
-
-    public OffsetMetadataTooLarge(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetOutOfRangeException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetOutOfRangeException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetOutOfRangeException.java
deleted file mode 100644
index dfc35b7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetOutOfRangeException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This offset is either larger or smaller than the range of offsets the server has for the given partition.
- * 
- */
-public class OffsetOutOfRangeException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public OffsetOutOfRangeException() {
-    }
-
-    public OffsetOutOfRangeException(String message) {
-        super(message);
-    }
-
-    public OffsetOutOfRangeException(Throwable cause) {
-        super(cause);
-    }
-
-    public OffsetOutOfRangeException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordBatchTooLargeException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordBatchTooLargeException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordBatchTooLargeException.java
deleted file mode 100644
index 360f042..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordBatchTooLargeException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This record batch is larger than the maximum allowable size
- */
-public class RecordBatchTooLargeException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public RecordBatchTooLargeException() {
-        super();
-    }
-
-    public RecordBatchTooLargeException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public RecordBatchTooLargeException(String message) {
-        super(message);
-    }
-
-    public RecordBatchTooLargeException(Throwable cause) {
-        super(cause);
-    }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordTooLargeException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordTooLargeException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordTooLargeException.java
deleted file mode 100644
index 0fd5a5f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordTooLargeException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This record is larger than the maximum allowable size
- */
-public class RecordTooLargeException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public RecordTooLargeException() {
-        super();
-    }
-
-    public RecordTooLargeException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public RecordTooLargeException(String message) {
-        super(message);
-    }
-
-    public RecordTooLargeException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RetriableException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RetriableException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RetriableException.java
deleted file mode 100644
index 419174f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RetriableException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A retryable exception is a transient exception that if retried may succeed.
- */
-public abstract class RetriableException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public RetriableException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public RetriableException(String message) {
-        super(message);
-    }
-
-    public RetriableException(Throwable cause) {
-        super(cause);
-    }
-
-    public RetriableException() {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/SerializationException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/SerializationException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/SerializationException.java
deleted file mode 100644
index 40f07fc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/SerializationException.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.kafka_backport.common.errors;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- *  Any exception during serialization in the producer
- */
-public class SerializationException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public SerializationException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public SerializationException(String message) {
-        super(message);
-    }
-
-    public SerializationException(Throwable cause) {
-        super(cause);
-    }
-
-    public SerializationException() {
-        super();
-    }
-
-    /* avoid the expensive and useless stack trace for serialization exceptions */
-    @Override
-    public Throwable fillInStackTrace() {
-        return this;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/TimeoutException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/TimeoutException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/TimeoutException.java
deleted file mode 100644
index 4fd5a32..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/TimeoutException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Indicates that a request timed out.
- */
-public class TimeoutException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public TimeoutException() {
-        super();
-    }
-
-    public TimeoutException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public TimeoutException(String message) {
-        super(message);
-    }
-
-    public TimeoutException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownConsumerIdException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownConsumerIdException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownConsumerIdException.java
deleted file mode 100644
index a86997c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownConsumerIdException.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class UnknownConsumerIdException extends RetriableException {
-    private static final long serialVersionUID = 1L;
-
-    public UnknownConsumerIdException() {
-        super();
-    }
-
-    public UnknownConsumerIdException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public UnknownConsumerIdException(String message) {
-        super(message);
-    }
-
-    public UnknownConsumerIdException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownServerException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownServerException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownServerException.java
deleted file mode 100644
index 423e8d3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownServerException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * An error occurred on the server for which the client doesn't have a corresponding error code. This is generally an
- * unexpected error.
- * 
- */
-public class UnknownServerException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public UnknownServerException() {
-    }
-
-    public UnknownServerException(String message) {
-        super(message);
-    }
-
-    public UnknownServerException(Throwable cause) {
-        super(cause);
-    }
-
-    public UnknownServerException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownTopicOrPartitionException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownTopicOrPartitionException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownTopicOrPartitionException.java
deleted file mode 100644
index 2b1a733..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownTopicOrPartitionException.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.kafka_backport.common.errors;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This topic/partition doesn't exist
- */
-public class UnknownTopicOrPartitionException extends InvalidMetadataException {
-
-    private static final long serialVersionUID = 1L;
-
-    public UnknownTopicOrPartitionException() {
-    }
-
-    public UnknownTopicOrPartitionException(String message) {
-        super(message);
-    }
-
-    public UnknownTopicOrPartitionException(Throwable throwable) {
-        super(throwable);
-    }
-
-    public UnknownTopicOrPartitionException(String message, Throwable throwable) {
-        super(message, throwable);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/CompoundStat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/CompoundStat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/CompoundStat.java
deleted file mode 100644
index 984e41e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/CompoundStat.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.metrics;
-
-import org.apache.flink.kafka_backport.common.MetricName;
-
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A compound stat is a stat where a single measurement and associated data structure feeds many metrics. This is the
- * example for a histogram which has many associated percentiles.
- */
-public interface CompoundStat extends Stat {
-
-    public List<NamedMeasurable> stats();
-
-    public static class NamedMeasurable {
-
-        private final MetricName name;
-        private final Measurable stat;
-
-        public NamedMeasurable(MetricName name, Measurable stat) {
-            super();
-            this.name = name;
-            this.stat = stat;
-        }
-
-        public MetricName name() {
-            return name;
-        }
-
-        public Measurable stat() {
-            return stat;
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/JmxReporter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/JmxReporter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/JmxReporter.java
deleted file mode 100644
index 5360efa..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/JmxReporter.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.metrics;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.management.Attribute;
-import javax.management.AttributeList;
-import javax.management.AttributeNotFoundException;
-import javax.management.DynamicMBean;
-import javax.management.InvalidAttributeValueException;
-import javax.management.JMException;
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanException;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.ReflectionException;
-import java.lang.management.ManagementFactory;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Register metrics in JMX as dynamic mbeans based on the metric names
- */
-public class JmxReporter implements MetricsReporter {
-
-    private static final Logger log = LoggerFactory.getLogger(JmxReporter.class);
-    private static final Object LOCK = new Object();
-    private String prefix;
-    private final Map<String, KafkaMbean> mbeans = new HashMap<String, KafkaMbean>();
-
-    public JmxReporter() {
-        this("");
-    }
-
-    /**
-     * Create a JMX reporter that prefixes all metrics with the given string.
-     */
-    public JmxReporter(String prefix) {
-        this.prefix = prefix;
-    }
-
-    @Override
-    public void configure(Map<String, ?> configs) {}
-
-    @Override
-    public void init(List<KafkaMetric> metrics) {
-        synchronized (LOCK) {
-            for (KafkaMetric metric : metrics)
-                addAttribute(metric);
-            for (KafkaMbean mbean : mbeans.values())
-                reregister(mbean);
-        }
-    }
-
-    @Override
-    public void metricChange(KafkaMetric metric) {
-        synchronized (LOCK) {
-            KafkaMbean mbean = addAttribute(metric);
-            reregister(mbean);
-        }
-    }
-
-    private KafkaMbean addAttribute(KafkaMetric metric) {
-        try {
-            MetricName metricName = metric.metricName();
-            String mBeanName = getMBeanName(metricName);
-            if (!this.mbeans.containsKey(mBeanName))
-                mbeans.put(mBeanName, new KafkaMbean(mBeanName));
-            KafkaMbean mbean = this.mbeans.get(mBeanName);
-            mbean.setAttribute(metricName.name(), metric);
-            return mbean;
-        } catch (JMException e) {
-            throw new KafkaException("Error creating mbean attribute for metricName :" + metric.metricName(), e);
-        }
-    }
-
-    /**
-     * @param metricName
-     * @return standard JMX MBean name in the following format domainName:type=metricType,key1=val1,key2=val2
-     */
-    private String getMBeanName(MetricName metricName) {
-        StringBuilder mBeanName = new StringBuilder();
-        mBeanName.append(prefix);
-        mBeanName.append(":type=");
-        mBeanName.append(metricName.group());
-        for (Map.Entry<String, String> entry : metricName.tags().entrySet()) {
-            if (entry.getKey().length() <= 0 || entry.getValue().length() <= 0)
-                continue;
-            mBeanName.append(",");
-            mBeanName.append(entry.getKey());
-            mBeanName.append("=");
-            mBeanName.append(entry.getValue());
-        }
-        return mBeanName.toString();
-    }
-
-    public void close() {
-        synchronized (LOCK) {
-            for (KafkaMbean mbean : this.mbeans.values())
-                unregister(mbean);
-        }
-    }
-
-    private void unregister(KafkaMbean mbean) {
-        MBeanServer server = ManagementFactory.getPlatformMBeanServer();
-        try {
-            if (server.isRegistered(mbean.name()))
-                server.unregisterMBean(mbean.name());
-        } catch (JMException e) {
-            throw new KafkaException("Error unregistering mbean", e);
-        }
-    }
-
-    private void reregister(KafkaMbean mbean) {
-        unregister(mbean);
-        try {
-            ManagementFactory.getPlatformMBeanServer().registerMBean(mbean, mbean.name());
-        } catch (JMException e) {
-            throw new KafkaException("Error registering mbean " + mbean.name(), e);
-        }
-    }
-
-    private static class KafkaMbean implements DynamicMBean {
-        private final ObjectName objectName;
-        private final Map<String, KafkaMetric> metrics;
-
-        public KafkaMbean(String mbeanName) throws MalformedObjectNameException {
-            this.metrics = new HashMap<String, KafkaMetric>();
-            this.objectName = new ObjectName(mbeanName);
-        }
-
-        public ObjectName name() {
-            return objectName;
-        }
-
-        public void setAttribute(String name, KafkaMetric metric) {
-            this.metrics.put(name, metric);
-        }
-
-        @Override
-        public Object getAttribute(String name) throws AttributeNotFoundException, MBeanException, ReflectionException {
-            if (this.metrics.containsKey(name))
-                return this.metrics.get(name).value();
-            else
-                throw new AttributeNotFoundException("Could not find attribute " + name);
-        }
-
-        @Override
-        public AttributeList getAttributes(String[] names) {
-            try {
-                AttributeList list = new AttributeList();
-                for (String name : names)
-                    list.add(new Attribute(name, getAttribute(name)));
-                return list;
-            } catch (Exception e) {
-                log.error("Error getting JMX attribute: ", e);
-                return new AttributeList();
-            }
-        }
-
-        @Override
-        public MBeanInfo getMBeanInfo() {
-            MBeanAttributeInfo[] attrs = new MBeanAttributeInfo[metrics.size()];
-            int i = 0;
-            for (Map.Entry<String, KafkaMetric> entry : this.metrics.entrySet()) {
-                String attribute = entry.getKey();
-                KafkaMetric metric = entry.getValue();
-                attrs[i] = new MBeanAttributeInfo(attribute,
-                                                  double.class.getName(),
-                                                  metric.metricName().description(),
-                                                  true,
-                                                  false,
-                                                  false);
-                i += 1;
-            }
-            return new MBeanInfo(this.getClass().getName(), "", attrs, null, null, null);
-        }
-
-        @Override
-        public Object invoke(String name, Object[] params, String[] sig) throws MBeanException, ReflectionException {
-            throw new UnsupportedOperationException("Set not allowed.");
-        }
-
-        @Override
-        public void setAttribute(Attribute attribute) throws AttributeNotFoundException,
-                InvalidAttributeValueException,
-                MBeanException,
-                ReflectionException {
-            throw new UnsupportedOperationException("Set not allowed.");
-        }
-
-        @Override
-        public AttributeList setAttributes(AttributeList list) {
-            throw new UnsupportedOperationException("Set not allowed.");
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/KafkaMetric.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/KafkaMetric.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/KafkaMetric.java
deleted file mode 100644
index 6245e79..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/KafkaMetric.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.metrics;
-
-import org.apache.flink.kafka_backport.common.Metric;
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.apache.flink.kafka_backport.common.utils.Time;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public final class KafkaMetric implements Metric {
-
-    private MetricName metricName;
-    private final Object lock;
-    private final Time time;
-    private final Measurable measurable;
-    private MetricConfig config;
-
-    KafkaMetric(Object lock, MetricName metricName, Measurable measurable, MetricConfig config, Time time) {
-        super();
-        this.metricName = metricName;
-        this.lock = lock;
-        this.measurable = measurable;
-        this.config = config;
-        this.time = time;
-    }
-
-    MetricConfig config() {
-        return this.config;
-    }
-
-    @Override
-    public MetricName metricName() {
-        return this.metricName;
-    }
-
-    @Override
-    public double value() {
-        synchronized (this.lock) {
-            return value(time.milliseconds());
-        }
-    }
-
-    double value(long timeMs) {
-        return this.measurable.measure(config, timeMs);
-    }
-
-    public void config(MetricConfig config) {
-        synchronized (lock) {
-            this.config = config;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Measurable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Measurable.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Measurable.java
deleted file mode 100644
index 08ed823..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Measurable.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.kafka_backport.common.metrics;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A measurable quantity that can be registered as a metric
- */
-public interface Measurable {
-
-    /**
-     * Measure this quantity and return the result as a double
-     * @param config The configuration for this metric
-     * @param now The POSIX time in milliseconds the measurement is being taken
-     * @return The measured value
-     */
-    public double measure(MetricConfig config, long now);
-
-}


[25/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Refactor, cleanup, and fix kafka consumers

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ValidatingExactlyOnceSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ValidatingExactlyOnceSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ValidatingExactlyOnceSink.java
new file mode 100644
index 0000000..a3c35fc
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ValidatingExactlyOnceSink.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.testutils;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+
+import java.util.BitSet;
+
+public class ValidatingExactlyOnceSink implements SinkFunction<Integer>, Checkpointed<Tuple2<Integer, BitSet>> {
+
+	private static final long serialVersionUID = 1748426382527469932L;
+	
+	private final int numElementsTotal;
+	
+	private BitSet duplicateChecker = new BitSet();  // this is checkpointed
+
+	private int numElements; // this is checkpointed
+
+	
+	public ValidatingExactlyOnceSink(int numElementsTotal) {
+		this.numElementsTotal = numElementsTotal;
+	}
+
+	
+	@Override
+	public void invoke(Integer value) throws Exception {
+		numElements++;
+		
+		if (duplicateChecker.get(value)) {
+			throw new Exception("Received a duplicate");
+		}
+		duplicateChecker.set(value);
+		
+		if (numElements == numElementsTotal) {
+			// validate
+			if (duplicateChecker.cardinality() != numElementsTotal) {
+				throw new Exception("Duplicate checker has wrong cardinality");
+			}
+			else if (duplicateChecker.nextClearBit(0) != numElementsTotal) {
+				throw new Exception("Received sparse sequence");
+			}
+			else {
+				throw new SuccessException();
+			}
+		}
+	}
+
+	@Override
+	public Tuple2<Integer, BitSet> snapshotState(long checkpointId, long checkpointTimestamp) {
+		return new Tuple2<>(numElements, duplicateChecker);
+	}
+
+	@Override
+	public void restoreState(Tuple2<Integer, BitSet> state) {
+		this.numElements = state.f0;
+		this.duplicateChecker = state.f1;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/resources/log4j-test.properties b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/resources/log4j-test.properties
index 9ede613..ded15e9 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/resources/log4j-test.properties
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/resources/log4j-test.properties
@@ -16,7 +16,7 @@
 # limitations under the License.
 ################################################################################
 
-log4j.rootLogger=INFO, testlogger
+log4j.rootLogger=INFo, testlogger
 
 log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
 log4j.appender.testlogger.target = System.err
@@ -24,4 +24,6 @@ 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
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
index 8655e0c..8c1883e 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
@@ -54,6 +54,7 @@ import org.apache.flink.api.common.functions.MapFunction;
 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.TypeInfoParser;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.net.NetUtils;
@@ -91,7 +92,7 @@ import scala.collection.Seq;
  * <p/>
  * https://github.com/sakserv/hadoop-mini-clusters (ASL licensed)
  */
-
+@SuppressWarnings("serial")
 public class KafkaITCase {
 
 	private static final Logger LOG = LoggerFactory.getLogger(KafkaITCase.class);
@@ -377,7 +378,12 @@ public class KafkaITCase {
 
 	private void readSequence(StreamExecutionEnvironment env, ConsumerConfig cc, final String topicName, final int valuesStartFrom, final int valuesCount, final int finalCount) throws Exception {
 		LOG.info("Reading sequence for verification until final count {}", finalCount);
-		TestPersistentKafkaSource<Tuple2<Integer, Integer>> pks = new TestPersistentKafkaSource<Tuple2<Integer, Integer>>(topicName, new TypeInformationSerializationSchema<Tuple2<Integer, Integer>>(new Tuple2<Integer, Integer>(1, 1), env.getConfig()), cc);
+
+		TypeInformation<Tuple2<Integer, Integer>> tuple2info = TypeInfoParser.parse("Tuple2<Integer, Integer>");
+		
+		TestPersistentKafkaSource<Tuple2<Integer, Integer>> pks = new TestPersistentKafkaSource<>(topicName, 
+				new TypeInformationSerializationSchema<>(tuple2info, env.getConfig()), cc);
+		
 		DataStream<Tuple2<Integer, Integer>> source = env.addSource(pks).map(new MapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>>() {
 			// we need to slow down the source so that it can participate in a few checkpoints.
 			// Otherwise it would write its data into buffers and shut down.
@@ -428,6 +434,9 @@ public class KafkaITCase {
 
 	private void writeSequence(StreamExecutionEnvironment env, String topicName, final int from, final int to) throws Exception {
 		LOG.info("Writing sequence from {} to {} to topic {}", from, to, topicName);
+
+		TypeInformation<Tuple2<Integer, Integer>> tuple2info = TypeInfoParser.parse("Tuple2<Integer, Integer>");
+		
 		DataStream<Tuple2<Integer, Integer>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
 			private static final long serialVersionUID = 1L;
 			boolean running = true;
@@ -457,7 +466,7 @@ public class KafkaITCase {
 		}).setParallelism(3);
 		stream.addSink(new KafkaSink<Tuple2<Integer, Integer>>(brokerConnectionStrings,
 				topicName,
-				new TypeInformationSerializationSchema<Tuple2<Integer, Integer>>(new Tuple2<Integer, Integer>(1, 1), env.getConfig()),
+				new TypeInformationSerializationSchema<>(tuple2info, env.getConfig()),
 				new T2Partitioner()
 		)).setParallelism(3);
 		env.execute("Write sequence from " + from + " to " + to + " to topic " + topicName);
@@ -472,6 +481,8 @@ public class KafkaITCase {
 			if(numPartitions != 3) {
 				throw new IllegalArgumentException("Expected three partitions");
 			}
+			
+			@SuppressWarnings("unchecked")
 			Tuple2<Integer, Integer> element = (Tuple2<Integer, Integer>) key;
 			return element.f0;
 		}
@@ -485,10 +496,14 @@ public class KafkaITCase {
 		String topic = "regularKafkaSourceTestTopic";
 		createTestTopic(topic, 1, 1);
 
+		TypeInformation<Tuple2<Long, String>> longStringInfo = TypeInfoParser.parse("Tuple2<Long, String>");
+
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 		// add consuming topology:
 		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(
-				new KafkaSource<Tuple2<Long, String>>(zookeeperConnectionString, topic, "myFlinkGroup", new TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig()), 5000));
+				new KafkaSource<Tuple2<Long, String>>(zookeeperConnectionString, topic, "myFlinkGroup",
+						new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig()), 5000));
+		
 		consuming.addSink(new SinkFunction<Tuple2<Long, String>>() {
 			private static final long serialVersionUID = 1L;
 
@@ -545,7 +560,8 @@ public class KafkaITCase {
 				running = false;
 			}
 		});
-		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, new TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig())));
+		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, 
+				new TypeInformationSerializationSchema<Tuple2<Long, String>>(longStringInfo, env.getConfig())));
 
 		tryExecute(env, "regular kafka source test");
 
@@ -559,12 +575,14 @@ public class KafkaITCase {
 		String topic = "tupleTestTopic";
 		createTestTopic(topic, 1, 1);
 
+		TypeInformation<Tuple2<Long, String>> longStringInfo = TypeInfoParser.parse("Tuple2<Long, String>");
+		
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 
 		// add consuming topology:
 		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(
-				new PersistentKafkaSource<Tuple2<Long, String>>(topic,
-						new TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig()),
+				new PersistentKafkaSource<>(topic,
+						new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig()),
 						standardCC
 				));
 		consuming.addSink(new RichSinkFunction<Tuple2<Long, String>>() {
@@ -631,7 +649,8 @@ public class KafkaITCase {
 				running = false;
 			}
 		});
-		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, new TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig())));
+		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, 
+				new TypeInformationSerializationSchema<Tuple2<Long, String>>(longStringInfo, env.getConfig())));
 
 		tryExecute(env, "tupletesttopology");
 
@@ -653,10 +672,14 @@ public class KafkaITCase {
 		String topic = "bigRecordTestTopic";
 		createTestTopic(topic, 1, 1);
 
+		final TypeInformation<Tuple2<Long, byte[]>> longBytesInfo = TypeInfoParser.parse("Tuple2<Long, byte[]>");
+		
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 
 		// add consuming topology:
-		TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema = new TypeInformationSerializationSchema<Tuple2<Long, byte[]>>(new Tuple2<Long, byte[]>(0L, new byte[]{0}), env.getConfig());
+		TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema = 
+				new TypeInformationSerializationSchema<Tuple2<Long, byte[]>>(longBytesInfo, env.getConfig());
+		
 		Properties consumerProps = new Properties();
 		consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 30));
 		consumerProps.setProperty("zookeeper.connect", zookeeperConnectionString);
@@ -734,7 +757,7 @@ public class KafkaITCase {
 		});
 
 		stream.addSink(new KafkaSink<Tuple2<Long, byte[]>>(brokerConnectionStrings, topic,
-						new TypeInformationSerializationSchema<Tuple2<Long, byte[]>>(new Tuple2<Long, byte[]>(0L, new byte[]{0}), env.getConfig()))
+						new TypeInformationSerializationSchema<Tuple2<Long, byte[]>>(longBytesInfo, env.getConfig()))
 		);
 
 		tryExecute(env, "big topology test");
@@ -751,12 +774,14 @@ public class KafkaITCase {
 
 		createTestTopic(topic, 3, 1);
 
+		final TypeInformation<Tuple2<Long, String>> longStringInfo = TypeInfoParser.parse("Tuple2<Long, String>");
+		
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 
 		// add consuming topology:
 		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(
 				new PersistentKafkaSource<Tuple2<Long, String>>(topic,
-						new TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig()),
+						new TypeInformationSerializationSchema<Tuple2<Long, String>>(longStringInfo, env.getConfig()),
 						standardCC));
 		consuming.addSink(new SinkFunction<Tuple2<Long, String>>() {
 			private static final long serialVersionUID = 1L;
@@ -829,7 +854,8 @@ public class KafkaITCase {
 				running = false;
 			}
 		});
-		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, new TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig()), new CustomPartitioner()));
+		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic,
+				new TypeInformationSerializationSchema<Tuple2<Long, String>>(longStringInfo, env.getConfig()), new CustomPartitioner()));
 
 		tryExecute(env, "custom partitioning test");
 
@@ -1138,7 +1164,6 @@ public class KafkaITCase {
 
 	/**
 	 * Read topic to list, only using Kafka code.
-	 * @return
 	 */
 	private static List<MessageAndMetadata<byte[], byte[]>> readTopicToList(String topicName, ConsumerConfig config, final int stopAfter) {
 		ConsumerConnector consumerConnector = Consumer.createJavaConsumerConnector(config);
@@ -1172,7 +1197,7 @@ public class KafkaITCase {
 		return result;
 	}
 
-	private static void printTopic(String topicName, ConsumerConfig config, DeserializationSchema deserializationSchema, int stopAfter){
+	private static void printTopic(String topicName, ConsumerConfig config, DeserializationSchema<?> deserializationSchema, int stopAfter){
 		List<MessageAndMetadata<byte[], byte[]>> contents = readTopicToList(topicName, config, stopAfter);
 		LOG.info("Printing contents of topic {} in consumer group {}", topicName, config.groupId());
 		for(MessageAndMetadata<byte[], byte[]> message: contents) {
@@ -1190,7 +1215,10 @@ public class KafkaITCase {
 		newProps.setProperty("zookeeper.connect", standardCC.zkConnect());
 
 		ConsumerConfig printerConfig = new ConsumerConfig(newProps);
-		DeserializationSchema deserializer = new TypeInformationSerializationSchema<Tuple2<Integer, Integer>>(new Tuple2<Integer, Integer>(1,1), ec);
+		TypeInformation<Tuple2<Integer, Integer>> typeInfo = TypeInfoParser.parse("Tuple2<Integer, Integer>");
+		
+		DeserializationSchema<Tuple2<Integer, Integer>> deserializer = 
+				new TypeInformationSerializationSchema<>(typeInfo, ec);
 		printTopic(topicName, printerConfig, deserializer, elements);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/util/UtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/util/UtilsTest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/util/UtilsTest.java
deleted file mode 100644
index 5c752ca..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/util/UtilsTest.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.util;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class UtilsTest {
-
-	/**
-	 * Ensure that the returned byte array has the expected size
-	 */
-	@Test
-	public void testTypeInformationSerializationSchema() {
-		final ExecutionConfig ec = new ExecutionConfig();
-
-		Tuple2<Integer, Integer> test = new Tuple2<Integer, Integer>(1,666);
-
-		TypeInformationSerializationSchema<Tuple2<Integer, Integer>> ser = new TypeInformationSerializationSchema<Tuple2<Integer, Integer>>(test, ec);
-
-		byte[] res = ser.serialize(test);
-		Assert.assertEquals(8, res.length);
-
-		Tuple2<Integer, Integer> another = ser.deserialize(res);
-		Assert.assertEquals(test.f0, another.f0);
-		Assert.assertEquals(test.f1, another.f1);
-	}
-
-	@Test
-	public void testGrowing() {
-		final ExecutionConfig ec = new ExecutionConfig();
-
-		Tuple2<Integer, byte[]> test1 = new Tuple2<Integer, byte[]>(1, new byte[16]);
-
-		TypeInformationSerializationSchema<Tuple2<Integer, byte[]>> ser = new TypeInformationSerializationSchema<Tuple2<Integer, byte[]>>(test1, ec);
-
-		byte[] res = ser.serialize(test1);
-		Assert.assertEquals(24, res.length);
-		Tuple2<Integer, byte[]> another = ser.deserialize(res);
-		Assert.assertEquals(16, another.f1.length);
-
-		test1 = new Tuple2<Integer, byte[]>(1, new byte[26]);
-
-		res = ser.serialize(test1);
-		Assert.assertEquals(34, res.length);
-		another = ser.deserialize(res);
-		Assert.assertEquals(26, another.f1.length);
-
-		test1 = new Tuple2<Integer, byte[]>(1, new byte[1]);
-
-		res = ser.serialize(test1);
-		Assert.assertEquals(9, res.length);
-		another = ser.deserialize(res);
-		Assert.assertEquals(1, another.f1.length);
-	}
-
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
index 0bcdb74..f38c557 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * BarrierBuffer continues receiving buffers from the blocked channels and stores them internally until 
  * the blocks are released.</p>
  */
-public class BarrierBuffer implements CheckpointBarrierHandler {
+public class BarrierBuffer implements CheckpointBarrierHandler, Runnable {
 
 	private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class);
 	
@@ -78,6 +78,17 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	private boolean endOfStream;
 
 
+	private int returnedBuffers;
+	
+	private int spilledBuffers;
+	
+	private int reReadBuffers;
+	
+	
+	private Thread debugPrinter;
+	
+	private volatile boolean printerRunning = true;
+	
 	/**
 	 * 
 	 * @param inputGate The input gate to draw the buffers and events from.
@@ -92,6 +103,10 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		
 		this.bufferSpiller = new BufferSpiller(ioManager, inputGate.getPageSize());
 		this.queuedBuffered = new ArrayDeque<BufferSpiller.SpilledBufferOrEventSequence>();
+		
+		this.debugPrinter = new Thread(this, "BB debugger");
+		this.debugPrinter.setDaemon(true);
+		this.debugPrinter.start();
 	}
 
 	// ------------------------------------------------------------------------
@@ -112,14 +127,21 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 					completeBufferedSequence();
 					return getNextNonBlocked();
 				}
+				else if (next.isBuffer()) {
+					reReadBuffers++;
+				}
 			}
 			
 			if (next != null) {
 				if (isBlocked(next.getChannelIndex())) {
 					// if the channel is blocked we, we just store the BufferOrEvent
 					bufferSpiller.add(next);
+					if (next.isBuffer()) {
+						spilledBuffers++;
+					}
 				}
 				else if (next.isBuffer()) {
+					returnedBuffers++;
 					return next;
 				}
 				else if (next.getEvent().getClass() == CheckpointBarrier.class) {
@@ -223,6 +245,9 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 
 	@Override
 	public void cleanup() throws IOException {
+		printerRunning = false;
+		debugPrinter.interrupt();
+		
 		bufferSpiller.close();
 		if (currentBuffered != null) {
 			currentBuffered.cleanup();
@@ -318,4 +343,21 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		return String.format("last checkpoint: %d, current barriers: %d, closed channels: %d",
 				currentCheckpointId, numBarriersReceived, numClosedChannels);
 	}
+	
+	// -------------------------------------
+	// TEMP HACK for debugging
+	
+	public void run() {
+		while (printerRunning) {
+			try {
+				Thread.sleep(5000);
+			}
+			catch (InterruptedException e) {
+				// ignore
+			}
+			
+			LOG.info("=====================> BARRIER BUFFER: returned buffers: {}, spilled buffers: {}, re-read buffers: {}",
+					returnedBuffers, spilledBuffers, reReadBuffers);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
index 9dd2b2d..6ff9712 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
@@ -50,17 +50,6 @@ public class TypeInformationSerializationSchema<T> implements DeserializationSch
 	// ------------------------------------------------------------------------
 
 	/**
-	 * Creates a new de-/serialization schema for the type of the given object instance.
-	 * The type will be passed through the {@link TypeExtractor} to create its type information.
-	 *
-	 * @param type A sample type instance for which the type information will be created.
-	 * @param ec The execution config, which is used to parametrize the type serializers.
-	 */
-	public TypeInformationSerializationSchema(T type, ExecutionConfig ec) {
-		this(TypeExtractor.getForObject(type), ec);
-	}
-
-	/**
 	 * Creates a new de-/serialization schema for the given type.
 	 * 
 	 * @param typeInfo The type information for the type de-/serialized by this schema.

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TypeInformationSerializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TypeInformationSerializationSchemaTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TypeInformationSerializationSchemaTest.java
index 28b26cb..8c847d3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TypeInformationSerializationSchemaTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TypeInformationSerializationSchemaTest.java
@@ -19,11 +19,11 @@
 package org.apache.flink.streaming.util;
 
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
+
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -77,20 +77,6 @@ public class TypeInformationSerializationSchemaTest {
 			fail(e.getMessage());
 		}
 	}
-
-	@Test
-	public void testCreateFromType() {
-		try {
-			TypeInformationSerializationSchema<String> schema = 
-					new TypeInformationSerializationSchema<String>("", new ExecutionConfig());
-			
-			assertEquals(BasicTypeInfo.STRING_TYPE_INFO, schema.getProducedType());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
 	
 	// ------------------------------------------------------------------------
 	//  Test data types


[10/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRecord.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRecord.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRecord.java
deleted file mode 100644
index 6b6c993..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRecord.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.kafka.copied.clients.consumer;
-
-/**
- * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the
- * record is being received and an offset that points to the record in a Kafka partition.
- */
-public final class ConsumerRecord<K, V> {
-    private final String topic;
-    private final int partition;
-    private final long offset;
-    private final K key;
-    private final V value;
-
-    /**
-     * Create a record with no key
-     * 
-     * @param topic The topic this record is received from
-     * @param partition The partition of the topic this record is received from
-     * @param offset The offset of this record in the corresponding Kafka partition
-     * @param value The record contents
-     */
-    public ConsumerRecord(String topic, int partition, long offset, K key, V value) {
-        if (topic == null)
-            throw new IllegalArgumentException("Topic cannot be null");
-        this.topic = topic;
-        this.partition = partition;
-        this.offset = offset;
-        this.key = key;
-        this.value = value;
-    }
-
-    /**
-     * The topic this record is received from
-     */
-    public String topic() {
-        return this.topic;
-    }
-
-    /**
-     * The partition from which this record is received
-     */
-    public int partition() {
-        return this.partition;
-    }
-
-    /**
-     * The key (or null if no key is specified)
-     */
-    public K key() {
-        return key;
-    }
-
-    /**
-     * The value
-     */
-    public V value() {
-        return value;
-    }
-
-    /**
-     * The position of this record in the corresponding Kafka partition.
-     */
-    public long offset() {
-        return offset;
-    }
-
-    @Override
-    public String toString() {
-        return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset()
-                + ", key = " + key + ", value = " + value + ")";
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRecords.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRecords.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRecords.java
deleted file mode 100644
index 70d781b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRecords.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.utils.AbstractIterator;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A container that holds the list {@link ConsumerRecord} per partition for a
- * particular topic. There is one for every topic returned by a
- * {@link Consumer#poll(long)} operation.
- */
-public class ConsumerRecords<K, V> implements Iterable<ConsumerRecord<K, V>> {
-    public static final ConsumerRecords<Object, Object> EMPTY =
-            new ConsumerRecords<Object, Object>(Collections.EMPTY_MAP);
-
-    private final Map<TopicPartition, List<ConsumerRecord<K, V>>> records;
-
-    public ConsumerRecords(Map<TopicPartition, List<ConsumerRecord<K, V>>> records) {
-        this.records = records;
-    }
-
-    /**
-     * Get just the records for the given partition
-     * 
-     * @param partition The partition to get records for
-     */
-    public Iterable<ConsumerRecord<K, V>> records(TopicPartition partition) {
-        List<ConsumerRecord<K, V>> recs = this.records.get(partition);
-        if (recs == null)
-            return Collections.emptyList();
-        else
-            return recs;
-    }
-
-    /**
-     * Get just the records for the given topic
-     */
-    public Iterable<ConsumerRecord<K, V>> records(String topic) {
-        if (topic == null)
-            throw new IllegalArgumentException("Topic must be non-null.");
-        List<List<ConsumerRecord<K, V>>> recs = new ArrayList<List<ConsumerRecord<K, V>>>();
-        for (Map.Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : records.entrySet()) {
-            if (entry.getKey().topic().equals(topic))
-                recs.add(entry.getValue());
-        }
-        return new ConcatenatedIterable<K, V>(recs);
-    }
-
-    @Override
-    public Iterator<ConsumerRecord<K, V>> iterator() {
-        return new ConcatenatedIterable<K, V>(records.values()).iterator();
-    }
-    
-    /**
-     * The number of records for all topics
-     */
-    public int count() {
-        int count = 0;
-        for (List<ConsumerRecord<K, V>> recs: this.records.values())
-            count += recs.size();
-        return count;
-    }
-
-    private static class ConcatenatedIterable<K, V> implements Iterable<ConsumerRecord<K, V>> {
-
-        private final Iterable<? extends Iterable<ConsumerRecord<K, V>>> iterables;
-
-        public ConcatenatedIterable(Iterable<? extends Iterable<ConsumerRecord<K, V>>> iterables) {
-            this.iterables = iterables;
-        }
-
-        @Override
-        public Iterator<ConsumerRecord<K, V>> iterator() {
-            return new AbstractIterator<ConsumerRecord<K, V>>() {
-                Iterator<? extends Iterable<ConsumerRecord<K, V>>> iters = iterables.iterator();
-                Iterator<ConsumerRecord<K, V>> current;
-
-                public ConsumerRecord<K, V> makeNext() {
-                    if (current == null || !current.hasNext()) {
-                        if (iters.hasNext())
-                            current = iters.next().iterator();
-                        else
-                            return allDone();
-                    }
-                    return current.next();
-                }
-            };
-        }
-    }
-
-    public boolean isEmpty() {
-        return records.isEmpty();
-    }
-
-    @SuppressWarnings("unchecked")
-    public static <K, V> ConsumerRecords<K, V> empty() {
-        return (ConsumerRecords<K, V>) EMPTY;
-    }
-
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/KafkaConsumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/KafkaConsumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/KafkaConsumer.java
deleted file mode 100644
index ea51f3a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/KafkaConsumer.java
+++ /dev/null
@@ -1,1121 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer;
-
-import org.apache.kafka.copied.clients.ClientUtils;
-import org.apache.kafka.copied.clients.Metadata;
-import org.apache.kafka.copied.clients.NetworkClient;
-import org.apache.kafka.copied.clients.consumer.internals.ConsumerNetworkClient;
-import org.apache.kafka.copied.clients.consumer.internals.Coordinator;
-import org.apache.kafka.copied.clients.consumer.internals.DelayedTask;
-import org.apache.kafka.copied.clients.consumer.internals.Fetcher;
-import org.apache.kafka.copied.clients.consumer.internals.SubscriptionState;
-import org.apache.kafka.copied.common.Cluster;
-import org.apache.kafka.copied.common.KafkaException;
-import org.apache.kafka.copied.common.Metric;
-import org.apache.kafka.copied.common.MetricName;
-import org.apache.kafka.copied.common.PartitionInfo;
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.metrics.JmxReporter;
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-import org.apache.kafka.copied.common.metrics.Metrics;
-import org.apache.kafka.copied.common.metrics.MetricsReporter;
-import org.apache.kafka.copied.common.network.Selector;
-import org.apache.kafka.copied.common.serialization.Deserializer;
-import org.apache.kafka.copied.common.utils.SystemTime;
-import org.apache.kafka.copied.common.utils.Time;
-import org.apache.kafka.copied.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetSocketAddress;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.ConcurrentModificationException;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.kafka.copied.common.utils.Utils.min;
-
-/**
- * A Kafka client that consumes records from a Kafka cluster.
- * <p>
- * It will transparently handle the failure of servers in the Kafka cluster, and transparently adapt as partitions of
- * data it subscribes to migrate within the cluster. This client also interacts with the server to allow groups of
- * consumers to load balance consumption using consumer groups (as described below).
- * <p>
- * The consumer maintains TCP connections to the necessary brokers to fetch data for the topics it subscribes to.
- * Failure to close the consumer after use will leak these connections.
- * <p>
- * The consumer is not thread-safe. See <a href="#multithreaded">Multi-threaded Processing</a> for more details.
- *
- * <h3>Offsets and Consumer Position</h3>
- * Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of
- * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer
- * which has position 5 has consumed records with offsets 0 through 4 and will next receive record with offset 5. There
- * are actually two notions of position relevant to the user of the consumer.
- * <p>
- * The {@link #position(TopicPartition) position} of the consumer gives the offset of the next record that will be given
- * out. It will be one larger than the highest offset the consumer has seen in that partition. It automatically advances
- * every time the consumer receives data calls {@link #poll(long)} and receives messages.
- * <p>
- * The {@link #commit(CommitType) committed position} is the last offset that has been saved securely. Should the
- * process fail and restart, this is the offset that it will recover to. The consumer can either automatically commit
- * offsets periodically, or it can choose to control this committed position manually by calling
- * {@link #commit(CommitType) commit}.
- * <p>
- * This distinction gives the consumer control over when a record is considered consumed. It is discussed in further
- * detail below.
- * 
- * <h3>Consumer Groups</h3>
- * 
- * Kafka uses the concept of <i>consumer groups</i> to allow a pool of processes to divide up the work of consuming and
- * processing records. These processes can either be running on the same machine or, as is more likely, they can be
- * distributed over many machines to provide additional scalability and fault tolerance for processing.
- * <p>
- * Each Kafka consumer must specify a consumer group that it belongs to. Kafka will deliver each message in the
- * subscribed topics to one process in each consumer group. This is achieved by balancing the partitions in the topic
- * over the consumer processes in each group. So if there is a topic with four partitions, and a consumer group with two
- * processes, each process would consume from two partitions. This group membership is maintained dynamically: if a
- * process fails the partitions assigned to it will be reassigned to other processes in the same group, and if a new
- * process joins the group, partitions will be moved from existing consumers to this new process.
- * <p>
- * So if two processes subscribe to a topic both specifying different groups they will each get all the records in that
- * topic; if they both specify the same group they will each get about half the records.
- * <p>
- * Conceptually you can think of a consumer group as being a single logical subscriber that happens to be made up of
- * multiple processes. As a multi-subscriber system, Kafka naturally supports having any number of consumer groups for a
- * given topic without duplicating data (additional consumers are actually quite cheap).
- * <p>
- * This is a slight generalization of the functionality that is common in messaging systems. To get semantics similar to
- * a queue in a traditional messaging system all processes would be part of a single consumer group and hence record
- * delivery would be balanced over the group like with a queue. Unlike a traditional messaging system, though, you can
- * have multiple such groups. To get semantics similar to pub-sub in a traditional messaging system each process would
- * have it's own consumer group, so each process would subscribe to all the records published to the topic.
- * <p>
- * In addition, when offsets are committed they are always committed for a given consumer group.
- * <p>
- * It is also possible for the consumer to manually specify the partitions it subscribes to, which disables this dynamic
- * partition balancing.
- * 
- * <h3>Usage Examples</h3>
- * The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to
- * demonstrate how to use them.
- * 
- * <h4>Simple Processing</h4>
- * This example demonstrates the simplest usage of Kafka's consumer api.
- * 
- * <pre>
- *     Properties props = new Properties();
- *     props.put(&quot;bootstrap.servers&quot;, &quot;localhost:9092&quot;);
- *     props.put(&quot;group.id&quot;, &quot;test&quot;);
- *     props.put(&quot;enable.auto.commit&quot;, &quot;true&quot;);
- *     props.put(&quot;auto.commit.interval.ms&quot;, &quot;1000&quot;);
- *     props.put(&quot;session.timeout.ms&quot;, &quot;30000&quot;);
- *     props.put(&quot;key.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
- *     props.put(&quot;value.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
- *     KafkaConsumer&lt;String, String&gt; consumer = new KafkaConsumer&lt;String, String&gt;(props);
- *     consumer.subscribe(&quot;foo&quot;, &quot;bar&quot;);
- *     while (true) {
- *         ConsumerRecords&lt;String, String&gt; records = consumer.poll(100);
- *         for (ConsumerRecord&lt;String, String&gt; record : records)
- *             System.out.printf(&quot;offset = %d, key = %s, value = %s&quot;, record.offset(), record.key(), record.value());
- *     }
- * </pre>
- * 
- * Setting <code>enable.auto.commit</code> means that offsets are committed automatically with a frequency controlled by
- * the config <code>auto.commit.interval.ms</code>.
- * <p>
- * The connection to the cluster is bootstrapped by specifying a list of one or more brokers to contact using the
- * configuration <code>bootstrap.servers</code>. This list is just used to discover the rest of the brokers in the
- * cluster and need not be an exhaustive list of servers in the cluster (though you may want to specify more than one in
- * case there are servers down when the client is connecting).
- * <p>
- * In this example the client is subscribing to the topics <i>foo</i> and <i>bar</i> as part of a group of consumers
- * called <i>test</i> as described above.
- * <p>
- * The broker will automatically detect failed processes in the <i>test</i> group by using a heartbeat mechanism. The
- * consumer will automatically ping the cluster periodically, which let's the cluster know that it is alive. As long as
- * the consumer is able to do this it is considered alive and retains the right to consume from the partitions assigned
- * to it. If it stops heartbeating for a period of time longer than <code>session.timeout.ms</code> then it will be
- * considered dead and it's partitions will be assigned to another process.
- * <p>
- * The deserializer settings specify how to turn bytes into objects. For example, by specifying string deserializers, we
- * are saying that our record's key and value will just be simple strings.
- * 
- * <h4>Controlling When Messages Are Considered Consumed</h4>
- * 
- * In this example we will consume a batch of records and batch them up in memory, when we have sufficient records
- * batched we will insert them into a database. If we allowed offsets to auto commit as in the previous example messages
- * would be considered consumed after they were given out by the consumer, and it would be possible that our process
- * could fail after we have read messages into our in-memory buffer but before they had been inserted into the database.
- * To avoid this we will manually commit the offsets only once the corresponding messages have been inserted into the
- * database. This gives us exact control of when a message is considered consumed. This raises the opposite possibility:
- * the process could fail in the interval after the insert into the database but before the commit (even though this
- * would likely just be a few milliseconds, it is a possibility). In this case the process that took over consumption
- * would consume from last committed offset and would repeat the insert of the last batch of data. Used in this way
- * Kafka provides what is often called "at-least once delivery" guarantees, as each message will likely be delivered one
- * time but in failure cases could be duplicated.
- * 
- * <pre>
- *     Properties props = new Properties();
- *     props.put(&quot;bootstrap.servers&quot;, &quot;localhost:9092&quot;);
- *     props.put(&quot;group.id&quot;, &quot;test&quot;);
- *     props.put(&quot;enable.auto.commit&quot;, &quot;false&quot;);
- *     props.put(&quot;auto.commit.interval.ms&quot;, &quot;1000&quot;);
- *     props.put(&quot;session.timeout.ms&quot;, &quot;30000&quot;);
- *     props.put(&quot;key.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
- *     props.put(&quot;value.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
- *     KafkaConsumer&lt;String, String&gt; consumer = new KafkaConsumer&lt;String, String&gt;(props);
- *     consumer.subscribe(&quot;foo&quot;, &quot;bar&quot;);
- *     int commitInterval = 200;
- *     List&lt;ConsumerRecord&lt;String, String&gt;&gt; buffer = new ArrayList&lt;ConsumerRecord&lt;String, String&gt;&gt;();
- *     while (true) {
- *         ConsumerRecords&lt;String, String&gt; records = consumer.poll(100);
- *         for (ConsumerRecord&lt;String, String&gt; record : records) {
- *             buffer.add(record);
- *             if (buffer.size() &gt;= commitInterval) {
- *                 insertIntoDb(buffer);
- *                 consumer.commit(CommitType.SYNC);
- *                 buffer.clear();
- *             }
- *         }
- *     }
- * </pre>
- * 
- * <h4>Subscribing To Specific Partitions</h4>
- * 
- * In the previous examples we subscribed to the topics we were interested in and let Kafka give our particular process
- * a fair share of the partitions for those topics. This provides a simple load balancing mechanism so multiple
- * instances of our program can divided up the work of processing records.
- * <p>
- * In this mode the consumer will just get the partitions it subscribes to and if the consumer instance fails no attempt
- * will be made to rebalance partitions to other instances.
- * <p>
- * There are several cases where this makes sense:
- * <ul>
- * <li>The first case is if the process is maintaining some kind of local state associated with that partition (like a
- * local on-disk key-value store) and hence it should only get records for the partition it is maintaining on disk.
- * <li>Another case is if the process itself is highly available and will be restarted if it fails (perhaps using a
- * cluster management framework like YARN, Mesos, or AWS facilities, or as part of a stream processing framework). In
- * this case there is no need for Kafka to detect the failure and reassign the partition, rather the consuming process
- * will be restarted on another machine.
- * </ul>
- * <p>
- * This mode is easy to specify, rather than subscribing to the topic, the consumer just subscribes to particular
- * partitions:
- * 
- * <pre>
- *     String topic = &quot;foo&quot;;
- *     TopicPartition partition0 = new TopicPartition(topic, 0);
- *     TopicPartition partition1 = new TopicPartition(topic, 1);
- *     consumer.subscribe(partition0);
- *     consumer.subscribe(partition1);
- * </pre>
- * 
- * The group that the consumer specifies is still used for committing offsets, but now the set of partitions will only
- * be changed if the consumer specifies new partitions, and no attempt at failure detection will be made.
- * <p>
- * It isn't possible to mix both subscription to specific partitions (with no load balancing) and to topics (with load
- * balancing) using the same consumer instance.
- * 
- * <h4>Managing Your Own Offsets</h4>
- * 
- * The consumer application need not use Kafka's built-in offset storage, it can store offsets in a store of it's own
- * choosing. The primary use case for this is allowing the application to store both the offset and the results of the
- * consumption in the same system in a way that both the results and offsets are stored atomically. This is not always
- * possible, but when it is it will make the consumption fully atomic and give "exactly once" semantics that are
- * stronger than the default "at-least once" semantics you get with Kafka's offset commit functionality.
- * <p>
- * Here are a couple of examples of this type of usage:
- * <ul>
- * <li>If the results of the consumption are being stored in a relational database, storing the offset in the database
- * as well can allow committing both the results and offset in a single transaction. Thus either the transaction will
- * succeed and the offset will be updated based on what was consumed or the result will not be stored and the offset
- * won't be updated.
- * <li>If the results are being stored in a local store it may be possible to store the offset there as well. For
- * example a search index could be built by subscribing to a particular partition and storing both the offset and the
- * indexed data together. If this is done in a way that is atomic, it is often possible to have it be the case that even
- * if a crash occurs that causes unsync'd data to be lost, whatever is left has the corresponding offset stored as well.
- * This means that in this case the indexing process that comes back having lost recent updates just resumes indexing
- * from what it has ensuring that no updates are lost.
- * </ul>
- * 
- * Each record comes with it's own offset, so to manage your own offset you just need to do the following:
- * <ol>
- * <li>Configure <code>enable.auto.commit=false</code>
- * <li>Use the offset provided with each {@link ConsumerRecord} to save your position.
- * <li>On restart restore the position of the consumer using {@link #seek(TopicPartition, long)}.
- * </ol>
- * 
- * This type of usage is simplest when the partition assignment is also done manually (this would be likely in the
- * search index use case described above). If the partition assignment is done automatically special care will also be
- * needed to handle the case where partition assignments change. This can be handled using a special callback specified
- * using <code>rebalance.callback.class</code>, which specifies an implementation of the interface
- * {@link ConsumerRebalanceCallback}. When partitions are taken from a consumer the consumer will want to commit its
- * offset for those partitions by implementing
- * {@link ConsumerRebalanceCallback#onPartitionsRevoked(Consumer, Collection)}. When partitions are assigned to a
- * consumer, the consumer will want to look up the offset for those new partitions an correctly initialize the consumer
- * to that position by implementing {@link ConsumerRebalanceCallback#onPartitionsAssigned(Consumer, Collection)}.
- * <p>
- * Another common use for {@link ConsumerRebalanceCallback} is to flush any caches the application maintains for
- * partitions that are moved elsewhere.
- * 
- * <h4>Controlling The Consumer's Position</h4>
- * 
- * In most use cases the consumer will simply consume records from beginning to end, periodically committing it's
- * position (either automatically or manually). However Kafka allows the consumer to manually control it's position,
- * moving forward or backwards in a partition at will. This means a consumer can re-consume older records, or skip to
- * the most recent records without actually consuming the intermediate records.
- * <p>
- * There are several instances where manually controlling the consumer's position can be useful.
- * <p>
- * One case is for time-sensitive record processing it may make sense for a consumer that falls far enough behind to not
- * attempt to catch up processing all records, but rather just skip to the most recent records.
- * <p>
- * Another use case is for a system that maintains local state as described in the previous section. In such a system
- * the consumer will want to initialize it's position on start-up to whatever is contained in the local store. Likewise
- * if the local state is destroyed (say because the disk is lost) the state may be recreated on a new machine by
- * reconsuming all the data and recreating the state (assuming that Kafka is retaining sufficient history).
- * 
- * Kafka allows specifying the position using {@link #seek(TopicPartition, long)} to specify the new position. Special
- * methods for seeking to the earliest and latest offset the server maintains are also available (
- * {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively).
- * 
- *
- * <h3><a name="multithreaded">Multi-threaded Processing</a></h3>
- * 
- * The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application
- * making the call. It is the responsibility of the user to ensure that multi-threaded access
- * is properly synchronized. Un-synchronized access will result in {@link ConcurrentModificationException}.
- *
- * <p>
- * The only exception to this rule is {@link #wakeup()}, which can safely be used from an external thread to
- * interrupt an active operation. In this case, a {@link ConsumerWakeupException} will be thrown from the thread
- * blocking on the operation. This can be used to shutdown the consumer from another thread. The following
- * snippet shows the typical pattern:
- *
- * <pre>
- * public class KafkaConsumerRunner implements Runnable {
- *     private final AtomicBoolean closed = new AtomicBoolean(false);
- *     private final KafkaConsumer consumer;
- *
- *     public void run() {
- *         try {
- *             consumer.subscribe("topic");
- *             while (!closed.get()) {
- *                 ConsumerRecords records = consumer.poll(10000);
- *                 // Handle new records
- *             }
- *         } catch (ConsumerWakeupException e) {
- *             // Ignore exception if closing
- *             if (!closed.get()) throw e;
- *         } finally {
- *             consumer.close();
- *         }
- *     }
- *
- *     // Shutdown hook which can be called from a separate thread
- *     public void shutdown() {
- *         closed.set(true);
- *         consumer.wakeup();
- *     }
- * }
- * </pre>
- *
- * Then in a separate thread, the consumer can be shutdown by setting the closed flag and waking up the consumer.
- *
- * <pre>
- *     closed.set(true);
- *     consumer.wakeup();
- * </pre>
- *
- * <p>
- * We have intentionally avoided implementing a particular threading model for processing. This leaves several
- * options for implementing multi-threaded processing of records.
- *
- * 
- * <h4>1. One Consumer Per Thread</h4>
- * 
- * A simple option is to give each thread it's own consumer instance. Here are the pros and cons of this approach:
- * <ul>
- * <li><b>PRO</b>: It is the easiest to implement
- * <li><b>PRO</b>: It is often the fastest as no inter-thread co-ordination is needed
- * <li><b>PRO</b>: It makes in-order processing on a per-partition basis very easy to implement (each thread just
- * processes messages in the order it receives them).
- * <li><b>CON</b>: More consumers means more TCP connections to the cluster (one per thread). In general Kafka handles
- * connections very efficiently so this is generally a small cost.
- * <li><b>CON</b>: Multiple consumers means more requests being sent to the server and slightly less batching of data
- * which can cause some drop in I/O throughput.
- * <li><b>CON</b>: The number of total threads across all processes will be limited by the total number of partitions.
- * </ul>
- * 
- * <h4>2. Decouple Consumption and Processing</h4>
- * 
- * Another alternative is to have one or more consumer threads that do all data consumption and hands off
- * {@link ConsumerRecords} instances to a blocking queue consumed by a pool of processor threads that actually handle
- * the record processing.
- * 
- * This option likewise has pros and cons:
- * <ul>
- * <li><b>PRO</b>: This option allows independently scaling the number of consumers and processors. This makes it
- * possible to have a single consumer that feeds many processor threads, avoiding any limitation on partitions.
- * <li><b>CON</b>: Guaranteeing order across the processors requires particular care as the threads will execute
- * independently an earlier chunk of data may actually be processed after a later chunk of data just due to the luck of
- * thread execution timing. For processing that has no ordering requirements this is not a problem.
- * <li><b>CON</b>: Manually committing the position becomes harder as it requires that all threads co-ordinate to ensure
- * that processing is complete for that partition.
- * </ul>
- * 
- * There are many possible variations on this approach. For example each processor thread can have it's own queue, and
- * the consumer threads can hash into these queues using the TopicPartition to ensure in-order consumption and simplify
- * commit.
- * 
- */
-public class KafkaConsumer<K, V> implements Consumer<K, V> {
-
-    private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class);
-    private static final long NO_CURRENT_THREAD = -1L;
-    private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1);
-
-    private final Coordinator coordinator;
-    private final Deserializer<K> keyDeserializer;
-    private final Deserializer<V> valueDeserializer;
-    private final Fetcher<K, V> fetcher;
-
-    private final Time time;
-    private final ConsumerNetworkClient client;
-    private final Metrics metrics;
-    private final SubscriptionState subscriptions;
-    private final Metadata metadata;
-    private final long retryBackoffMs;
-    private final boolean autoCommit;
-    private final long autoCommitIntervalMs;
-    private boolean closed = false;
-
-    // currentThread holds the threadId of the current thread accessing KafkaConsumer
-    // and is used to prevent multi-threaded access
-    private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD);
-    // refcount is used to allow reentrant access by the thread who has acquired currentThread
-    private final AtomicInteger refcount = new AtomicInteger(0);
-
-    // TODO: This timeout controls how long we should wait before retrying a request. We should be able
-    //       to leverage the work of KAFKA-2120 to get this value from configuration.
-    private long requestTimeoutMs = 5000L;
-
-    /**
-     * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
-     * are documented <a href="http://kafka.apache.org/documentation.html#consumerconfigs" >here</a>. Values can be
-     * either strings or objects of the appropriate type (for example a numeric configuration would accept either the
-     * string "42" or the integer 42).
-     * <p>
-     * Valid configuration strings are documented at {@link ConsumerConfig}
-     * 
-     * @param configs The consumer configs
-     */
-    public KafkaConsumer(Map<String, Object> configs) {
-        this(configs, null, null, null);
-    }
-
-    /**
-     * A consumer is instantiated by providing a set of key-value pairs as configuration, a
-     * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}.
-     * <p>
-     * Valid configuration strings are documented at {@link ConsumerConfig}
-     * 
-     * @param configs The consumer configs
-     * @param callback A callback interface that the user can implement to manage customized offsets on the start and
-     *            end of every rebalance operation.
-     * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method
-     *            won't be called in the consumer when the deserializer is passed in directly.
-     * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method
-     *            won't be called in the consumer when the deserializer is passed in directly.
-     */
-    public KafkaConsumer(Map<String, Object> configs,
-                         ConsumerRebalanceCallback callback,
-                         Deserializer<K> keyDeserializer,
-                         Deserializer<V> valueDeserializer) {
-        this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)),
-            callback,
-            keyDeserializer,
-            valueDeserializer);
-    }
-
-    /**
-     * A consumer is instantiated by providing a {@link Properties} object as configuration. Valid
-     * configuration strings are documented at {@link ConsumerConfig} A consumer is instantiated by providing a
-     * {@link Properties} object as configuration. Valid configuration strings are documented at
-     * {@link ConsumerConfig}
-     */
-    public KafkaConsumer(Properties properties) {
-        this(properties, null, null, null);
-    }
-
-    /**
-     * A consumer is instantiated by providing a {@link Properties} object as configuration and a
-     * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}.
-     * <p>
-     * Valid configuration strings are documented at {@link ConsumerConfig}
-     * 
-     * @param properties The consumer configuration properties
-     * @param callback A callback interface that the user can implement to manage customized offsets on the start and
-     *            end of every rebalance operation.
-     * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method
-     *            won't be called in the consumer when the deserializer is passed in directly.
-     * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method
-     *            won't be called in the consumer when the deserializer is passed in directly.
-     */
-    public KafkaConsumer(Properties properties,
-                         ConsumerRebalanceCallback callback,
-                         Deserializer<K> keyDeserializer,
-                         Deserializer<V> valueDeserializer) {
-        this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)),
-             callback,
-             keyDeserializer,
-             valueDeserializer);
-    }
-
-    @SuppressWarnings("unchecked")
-    private KafkaConsumer(ConsumerConfig config,
-                          ConsumerRebalanceCallback callback,
-                          Deserializer<K> keyDeserializer,
-                          Deserializer<V> valueDeserializer) {
-        try {
-            log.debug("Starting the Kafka consumer");
-            if (callback == null)
-                callback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
-                        ConsumerRebalanceCallback.class);
-            this.time = new SystemTime();
-            this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
-            this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
-
-            MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
-                    .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
-                            TimeUnit.MILLISECONDS);
-            String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
-            String jmxPrefix = "kafka.consumer";
-            if (clientId.length() <= 0)
-                clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement();
-            List<MetricsReporter> reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG,
-                    MetricsReporter.class);
-            reporters.add(new JmxReporter(jmxPrefix));
-            this.metrics = new Metrics(metricConfig, reporters, time);
-            this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
-            this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG));
-            List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
-            this.metadata.update(Cluster.bootstrap(addresses), 0);
-
-            String metricGrpPrefix = "consumer";
-            Map<String, String> metricsTags = new LinkedHashMap<String, String>();
-            metricsTags.put("client-id", clientId);
-            NetworkClient netClient = new NetworkClient(
-                    new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags),
-                    this.metadata,
-                    clientId,
-                    100, // a fixed large enough value will suffice
-                    config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG),
-                    config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG),
-                    config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG));
-            this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs);
-            OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase());
-            this.subscriptions = new SubscriptionState(offsetResetStrategy);
-            this.coordinator = new Coordinator(this.client,
-                    config.getString(ConsumerConfig.GROUP_ID_CONFIG),
-                    config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG),
-                    config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
-                    this.subscriptions,
-                    metrics,
-                    metricGrpPrefix,
-                    metricsTags,
-                    this.time,
-                    requestTimeoutMs,
-                    retryBackoffMs,
-                    wrapRebalanceCallback(callback));
-            if (keyDeserializer == null) {
-                this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
-                        Deserializer.class);
-                this.keyDeserializer.configure(config.originals(), true);
-            } else {
-                this.keyDeserializer = keyDeserializer;
-            }
-            if (valueDeserializer == null) {
-                this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
-                        Deserializer.class);
-                this.valueDeserializer.configure(config.originals(), false);
-            } else {
-                this.valueDeserializer = valueDeserializer;
-            }
-            this.fetcher = new Fetcher<K, V>(this.client,
-                    config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG),
-                    config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG),
-                    config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG),
-                    config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG),
-                    this.keyDeserializer,
-                    this.valueDeserializer,
-                    this.metadata,
-                    this.subscriptions,
-                    metrics,
-                    metricGrpPrefix,
-                    metricsTags,
-                    this.time,
-                    this.retryBackoffMs);
-
-            config.logUnused();
-
-            if (autoCommit)
-                scheduleAutoCommitTask(autoCommitIntervalMs);
-
-            log.debug("Kafka consumer created");
-        } catch (Throwable t) {
-            // call close methods if internal objects are already constructed
-            // this is to prevent resource leak. see KAFKA-2121
-            close(true);
-            // now propagate the exception
-            throw new KafkaException("Failed to construct kafka consumer", t);
-        }
-    }
-
-    /**
-     * The set of partitions currently assigned to this consumer. If subscription happened by directly subscribing to
-     * partitions using {@link #subscribe(TopicPartition...)} then this will simply return the list of partitions that
-     * were subscribed to. If subscription was done by specifying only the topic using {@link #subscribe(String...)}
-     * then this will give the set of topics currently assigned to the consumer (which may be none if the assignment
-     * hasn't happened yet, or the partitions are in the process of getting reassigned).
-     */
-    public Set<TopicPartition> subscriptions() {
-        acquire();
-        try {
-            return Collections.unmodifiableSet(this.subscriptions.assignedPartitions());
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Incrementally subscribes to the given list of topics and uses the consumer's group management functionality
-     * <p>
-     * As part of group management, the consumer will keep track of the list of consumers that belong to a particular
-     * group and will trigger a rebalance operation if one of the following events trigger -
-     * <ul>
-     * <li>Number of partitions change for any of the subscribed list of topics
-     * <li>Topic is created or deleted
-     * <li>An existing member of the consumer group dies
-     * <li>A new member is added to an existing consumer group via the join API
-     * </ul>
-     * 
-     * @param topics A variable list of topics that the consumer wants to subscribe to
-     */
-    @Override
-    public void subscribe(String... topics) {
-        acquire();
-        try {
-            log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", "));
-            for (String topic : topics)
-                this.subscriptions.subscribe(topic);
-            metadata.addTopics(topics);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Incrementally subscribes to a specific topic partition and does not use the consumer's group management
-     * functionality. As such, there will be no rebalance operation triggered when group membership or cluster and topic
-     * metadata change.
-     * <p>
-     *
-     * @param partitions Partitions to incrementally subscribe to
-     */
-    @Override
-    public void subscribe(TopicPartition... partitions) {
-        acquire();
-        try {
-            log.debug("Subscribed to partitions(s): {}", Utils.join(partitions, ", "));
-            for (TopicPartition tp : partitions) {
-                this.subscriptions.subscribe(tp);
-                metadata.addTopics(tp.topic());
-            }
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Unsubscribe from the specific topics. This will trigger a rebalance operation and records for this topic will not
-     * be returned from the next {@link #poll(long) poll()} onwards
-     * 
-     * @param topics Topics to unsubscribe from
-     */
-    public void unsubscribe(String... topics) {
-        acquire();
-        try {
-            log.debug("Unsubscribed from topic(s): {}", Utils.join(topics, ", "));
-            // throw an exception if the topic was never subscribed to
-            for (String topic : topics)
-                this.subscriptions.unsubscribe(topic);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Unsubscribe from the specific topic partitions. records for these partitions will not be returned from the next
-     * {@link #poll(long) poll()} onwards
-     * 
-     * @param partitions Partitions to unsubscribe from
-     */
-    public void unsubscribe(TopicPartition... partitions) {
-        acquire();
-        try {
-            log.debug("Unsubscribed from partitions(s): {}", Utils.join(partitions, ", "));
-            // throw an exception if the partition was never subscribed to
-            for (TopicPartition partition : partitions)
-                this.subscriptions.unsubscribe(partition);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Fetches data for the topics or partitions specified using one of the subscribe APIs. It is an error to not have
-     * subscribed to any topics or partitions before polling for data.
-     * <p>
-     * The offset used for fetching the data is governed by whether or not {@link #seek(TopicPartition, long)} is used.
-     * If {@link #seek(TopicPartition, long)} is used, it will use the specified offsets on startup and on every
-     * rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed
-     * offset using {@link #commit(Map, CommitType) commit(offsets, sync)} for the subscribed list of partitions.
-     * 
-     * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, returns
-     *            immediately with any records available now. Must not be negative.
-     * @return map of topic to records since the last fetch for the subscribed list of topics and partitions
-     * 
-     * @throws NoOffsetForPartitionException If there is no stored offset for a subscribed partition and no automatic
-     *             offset reset policy has been configured.
-     */
-    @Override
-    public ConsumerRecords<K, V> poll(long timeout) {
-        acquire();
-        try {
-            if (timeout < 0)
-                throw new IllegalArgumentException("Timeout must not be negative");
-
-            // poll for new data until the timeout expires
-            long remaining = timeout;
-            while (remaining >= 0) {
-                long start = time.milliseconds();
-                Map<TopicPartition, List<ConsumerRecord<K, V>>> records = pollOnce(remaining);
-                long end = time.milliseconds();
-
-                if (!records.isEmpty()) {
-                    // if data is available, then return it, but first send off the
-                    // next round of fetches to enable pipelining while the user is
-                    // handling the fetched records.
-                    fetcher.initFetches(metadata.fetch());
-                    client.poll(0);
-                    return new ConsumerRecords<K, V>(records);
-                }
-
-                remaining -= end - start;
-
-                // nothing was available, so we should backoff before retrying
-                if (remaining > 0) {
-                    Utils.sleep(min(remaining, retryBackoffMs));
-                    remaining -= time.milliseconds() - end;
-                }
-            }
-
-            return ConsumerRecords.empty();
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Do one round of polling. In addition to checking for new data, this does any needed
-     * heart-beating, auto-commits, and offset updates.
-     * @param timeout The maximum time to block in the underlying poll
-     * @return The fetched records (may be empty)
-     */
-    private Map<TopicPartition, List<ConsumerRecord<K, V>>> pollOnce(long timeout) {
-        // TODO: Sub-requests should take into account the poll timeout (KAFKA-1894)
-        coordinator.ensureCoordinatorKnown();
-
-        // ensure we have partitions assigned if we expect to
-        if (subscriptions.partitionsAutoAssigned())
-            coordinator.ensurePartitionAssignment();
-
-        // fetch positions if we have partitions we're subscribed to that we
-        // don't know the offset for
-        if (!subscriptions.hasAllFetchPositions())
-            updateFetchPositions(this.subscriptions.missingFetchPositions());
-
-        // init any new fetches (won't resend pending fetches)
-        Cluster cluster = this.metadata.fetch();
-        fetcher.initFetches(cluster);
-        client.poll(timeout);
-        return fetcher.fetchedRecords();
-    }
-
-    private void scheduleAutoCommitTask(final long interval) {
-        DelayedTask task = new DelayedTask() {
-            public void run(long now) {
-                commit(CommitType.ASYNC);
-                client.schedule(this, now + interval);
-            }
-        };
-        client.schedule(task, time.milliseconds() + interval);
-    }
-
-    /**
-     * Commits the specified offsets for the specified list of topics and partitions to Kafka.
-     * <p>
-     * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every
-     * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
-     * should not be used.
-     * <p>
-     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
-     * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use
-     * {@link #commit(Map, CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC})
-     * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown
-     * to the caller).
-     *
-     * @param offsets The list of offsets per partition that should be committed to Kafka.
-     * @param commitType Control whether the commit is blocking
-     */
-    @Override
-    public void commit(final Map<TopicPartition, Long> offsets, CommitType commitType) {
-        commit(offsets, commitType, null);
-    }
-
-    /**
-     * Commits the specified offsets for the specified list of topics and partitions to Kafka.
-     * <p>
-     * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every
-     * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
-     * should not be used.
-     * <p>
-     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
-     * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e.
-     * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In
-     * this case, the error is either passed to the callback (if provided) or thrown to the caller.
-     *
-     * @param offsets The list of offsets per partition that should be committed to Kafka.
-     * @param commitType Control whether the commit is blocking
-     * @param callback Callback to invoke when the commit completes
-     */
-    @Override
-    public void commit(final Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
-        acquire();
-        try {
-            log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets);
-            coordinator.commitOffsets(offsets, commitType, callback);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions.
-     * <p>
-     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
-     * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
-     * should not be used.
-     * <p>
-     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
-     * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e.
-     * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In
-     * this case, the error is either passed to the callback (if provided) or thrown to the caller.
-     *
-     * @param commitType Whether or not the commit should block until it is acknowledged.
-     * @param callback Callback to invoke when the commit completes
-     */
-    @Override
-    public void commit(CommitType commitType, ConsumerCommitCallback callback) {
-        acquire();
-        try {
-            // need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance)
-            Map<TopicPartition, Long> allConsumed = new HashMap<TopicPartition, Long>(this.subscriptions.allConsumed());
-            commit(allConsumed, commitType, callback);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions.
-     * <p>
-     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
-     * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
-     * should not be used.
-     * <p>
-     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
-     * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use
-     * {@link #commit(CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC})
-     * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown
-     * to the caller).
-     *
-     * @param commitType Whether or not the commit should block until it is acknowledged.
-     */
-    @Override
-    public void commit(CommitType commitType) {
-        commit(commitType, null);
-    }
-
-    /**
-     * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API
-     * is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that
-     * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets
-     */
-    @Override
-    public void seek(TopicPartition partition, long offset) {
-        acquire();
-        try {
-            log.debug("Seeking to offset {} for partition {}", offset, partition);
-            this.subscriptions.seek(partition, offset);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Seek to the first offset for each of the given partitions
-     */
-    public void seekToBeginning(TopicPartition... partitions) {
-        acquire();
-        try {
-            Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
-                    : Arrays.asList(partitions);
-            for (TopicPartition tp : parts) {
-                log.debug("Seeking to beginning of partition {}", tp);
-                subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST);
-            }
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Seek to the last offset for each of the given partitions
-     */
-    public void seekToEnd(TopicPartition... partitions) {
-        acquire();
-        try {
-            Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
-                    : Arrays.asList(partitions);
-            for (TopicPartition tp : parts) {
-                log.debug("Seeking to end of partition {}", tp);
-                subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST);
-            }
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Returns the offset of the <i>next record</i> that will be fetched (if a record with that offset exists).
-     * 
-     * @param partition The partition to get the position for
-     * @return The offset
-     * @throws NoOffsetForPartitionException If a position hasn't been set for a given partition, and no reset policy is
-     *             available.
-     */
-    public long position(TopicPartition partition) {
-        acquire();
-        try {
-            if (!this.subscriptions.assignedPartitions().contains(partition))
-                throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer.");
-            Long offset = this.subscriptions.consumed(partition);
-            if (offset == null) {
-                updateFetchPositions(Collections.singleton(partition));
-                return this.subscriptions.consumed(partition);
-            } else {
-                return offset;
-            }
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Fetches the last committed offset for the given partition (whether the commit happened by this process or
-     * another). This offset will be used as the position for the consumer in the event of a failure.
-     * <p>
-     * This call may block to do a remote call if the partition in question isn't assigned to this consumer or if the
-     * consumer hasn't yet initialized it's cache of committed offsets.
-     * 
-     * @param partition The partition to check
-     * @return The last committed offset or null if no offset has been committed
-     * @throws NoOffsetForPartitionException If no offset has ever been committed by any process for the given
-     *             partition.
-     */
-    @Override
-    public long committed(TopicPartition partition) {
-        acquire();
-        try {
-            Long committed;
-            if (subscriptions.assignedPartitions().contains(partition)) {
-                committed = this.subscriptions.committed(partition);
-                if (committed == null) {
-                    coordinator.refreshCommittedOffsetsIfNeeded();
-                    committed = this.subscriptions.committed(partition);
-                }
-            } else {
-                Map<TopicPartition, Long> offsets = coordinator.fetchCommittedOffsets(Collections.singleton(partition));
-                committed = offsets.get(partition);
-            }
-
-            if (committed == null)
-                throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition);
-
-            return committed;
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Get the metrics kept by the consumer
-     */
-    @Override
-    public Map<MetricName, ? extends Metric> metrics() {
-        return Collections.unmodifiableMap(this.metrics.metrics());
-    }
-
-    /**
-     * Get metadata about the partitions for a given topic. This method will issue a remote call to the server if it
-     * does not already have any metadata about the given topic.
-     * 
-     * @param topic The topic to get partition metadata for
-     * @return The list of partitions
-     */
-    @Override
-    public List<PartitionInfo> partitionsFor(String topic) {
-        acquire();
-        try {
-            Cluster cluster = this.metadata.fetch();
-            List<PartitionInfo> parts = cluster.partitionsForTopic(topic);
-            if (parts == null) {
-                metadata.add(topic);
-                client.awaitMetadataUpdate();
-                parts = metadata.fetch().partitionsForTopic(topic);
-            }
-            return parts;
-        } finally {
-            release();
-        }
-    }
-
-    @Override
-    public void close() {
-        acquire();
-        try {
-            if (closed) return;
-            close(false);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Wakeup the consumer. This method is thread-safe and is useful in particular to abort a long poll.
-     * The thread which is blocking in an operation will throw {@link ConsumerWakeupException}.
-     */
-    @Override
-    public void wakeup() {
-        this.client.wakeup();
-    }
-
-    private void close(boolean swallowException) {
-        log.trace("Closing the Kafka consumer.");
-        AtomicReference<Throwable> firstException = new AtomicReference<Throwable>();
-        this.closed = true;
-        ClientUtils.closeQuietly(metrics, "consumer metrics", firstException);
-        ClientUtils.closeQuietly(client, "consumer network client", firstException);
-        ClientUtils.closeQuietly(keyDeserializer, "consumer key deserializer", firstException);
-        ClientUtils.closeQuietly(valueDeserializer, "consumer value deserializer", firstException);
-        log.debug("The Kafka consumer has closed.");
-        if (firstException.get() != null && !swallowException) {
-            throw new KafkaException("Failed to close kafka consumer", firstException.get());
-        }
-    }
-
-    private Coordinator.RebalanceCallback wrapRebalanceCallback(final ConsumerRebalanceCallback callback) {
-        return new Coordinator.RebalanceCallback() {
-            @Override
-            public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
-                callback.onPartitionsAssigned(KafkaConsumer.this, partitions);
-            }
-
-            @Override
-            public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
-                callback.onPartitionsRevoked(KafkaConsumer.this, partitions);
-            }
-        };
-    }
-
-    /**
-     * Set the fetch position to the committed position (if there is one)
-     * or reset it using the offset reset policy the user has configured.
-     *
-     * @param partitions The partitions that needs updating fetch positions
-     * @throws NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is
-     *             defined
-     */
-    private void updateFetchPositions(Set<TopicPartition> partitions) {
-        // refresh commits for all assigned partitions
-        coordinator.refreshCommittedOffsetsIfNeeded();
-
-        // then do any offset lookups in case some positions are not known
-        fetcher.updateFetchPositions(partitions);
-    }
-
-    /*
-     * Check that the consumer hasn't been closed.
-     */
-    private void ensureNotClosed() {
-        if (this.closed)
-            throw new IllegalStateException("This consumer has already been closed.");
-    }
-
-    /**
-     * Acquire the light lock protecting this consumer from multi-threaded access. Instead of blocking
-     * when the lock is not available, however, we just throw an exception (since multi-threaded usage is not
-     * supported).
-     * @throws IllegalStateException if the consumer has been closed
-     * @throws ConcurrentModificationException if another thread already has the lock
-     */
-    private void acquire() {
-        ensureNotClosed();
-        long threadId = Thread.currentThread().getId();
-        if (threadId != currentThread.get() && !currentThread.compareAndSet(NO_CURRENT_THREAD, threadId))
-            throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access");
-        refcount.incrementAndGet();
-    }
-
-    /**
-     * Release the light lock protecting the consumer from multi-threaded access.
-     */
-    private void release() {
-        if (refcount.decrementAndGet() == 0)
-            currentThread.set(NO_CURRENT_THREAD);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/MockConsumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/MockConsumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/MockConsumer.java
deleted file mode 100644
index ce94cea..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/MockConsumer.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer;
-
-import org.apache.kafka.copied.clients.consumer.internals.SubscriptionState;
-import org.apache.kafka.copied.common.Metric;
-import org.apache.kafka.copied.common.MetricName;
-import org.apache.kafka.copied.common.PartitionInfo;
-import org.apache.kafka.copied.common.TopicPartition;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-/**
- * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. This class is <i> not
- * threadsafe </i>
- * <p>
- * The consumer runs in the user thread and multiplexes I/O over TCP connections to each of the brokers it needs to
- * communicate with. Failure to close the consumer after use will leak these resources.
- */
-public class MockConsumer<K, V> implements Consumer<K, V> {
-
-    private final Map<String, List<PartitionInfo>> partitions;
-    private final SubscriptionState subscriptions;
-    private Map<TopicPartition, List<ConsumerRecord<K, V>>> records;
-    private boolean closed;
-
-    public MockConsumer(OffsetResetStrategy offsetResetStrategy) {
-        this.subscriptions = new SubscriptionState(offsetResetStrategy);
-        this.partitions = new HashMap<String, List<PartitionInfo>>();
-        this.records = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
-        this.closed = false;
-    }
-    
-    @Override
-    public synchronized Set<TopicPartition> subscriptions() {
-        return this.subscriptions.assignedPartitions();
-    }
-
-    @Override
-    public synchronized void subscribe(String... topics) {
-        ensureNotClosed();
-        for (String topic : topics)
-            this.subscriptions.subscribe(topic);
-    }
-
-    @Override
-    public synchronized void subscribe(TopicPartition... partitions) {
-        ensureNotClosed();
-        for (TopicPartition partition : partitions)
-            this.subscriptions.subscribe(partition);
-    }
-
-    public synchronized void unsubscribe(String... topics) {
-        ensureNotClosed();
-        for (String topic : topics)
-            this.subscriptions.unsubscribe(topic);
-    }
-
-    public synchronized void unsubscribe(TopicPartition... partitions) {
-        ensureNotClosed();
-        for (TopicPartition partition : partitions)
-            this.subscriptions.unsubscribe(partition);
-    }
-
-    @Override
-    public synchronized ConsumerRecords<K, V> poll(long timeout) {
-        ensureNotClosed();
-        // update the consumed offset
-        for (Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : this.records.entrySet()) {
-            List<ConsumerRecord<K, V>> recs = entry.getValue();
-            if (!recs.isEmpty())
-                this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset());
-        }
-
-        ConsumerRecords<K, V> copy = new ConsumerRecords<K, V>(this.records);
-        this.records = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
-        return copy;
-    }
-
-    public synchronized void addRecord(ConsumerRecord<K, V> record) {
-        ensureNotClosed();
-        TopicPartition tp = new TopicPartition(record.topic(), record.partition());
-        this.subscriptions.assignedPartitions().add(tp);
-        List<ConsumerRecord<K, V>> recs = this.records.get(tp);
-        if (recs == null) {
-            recs = new ArrayList<ConsumerRecord<K, V>>();
-            this.records.put(tp, recs);
-        }
-        recs.add(record);
-    }
-
-    @Override
-    public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
-        ensureNotClosed();
-        for (Entry<TopicPartition, Long> entry : offsets.entrySet())
-            subscriptions.committed(entry.getKey(), entry.getValue());
-        if (callback != null) {
-            callback.onComplete(offsets, null);
-        }
-    }
-
-    @Override
-    public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType) {
-        commit(offsets, commitType, null);
-    }
-
-    @Override
-    public synchronized void commit(CommitType commitType, ConsumerCommitCallback callback) {
-        ensureNotClosed();
-        commit(this.subscriptions.allConsumed(), commitType, callback);
-    }
-
-    @Override
-    public synchronized void commit(CommitType commitType) {
-        commit(commitType, null);
-    }
-
-    @Override
-    public synchronized void seek(TopicPartition partition, long offset) {
-        ensureNotClosed();
-        subscriptions.seek(partition, offset);
-    }
-
-    @Override
-    public synchronized long committed(TopicPartition partition) {
-        ensureNotClosed();
-        return subscriptions.committed(partition);
-    }
-
-    @Override
-    public synchronized long position(TopicPartition partition) {
-        ensureNotClosed();
-        return subscriptions.consumed(partition);
-    }
-
-    @Override
-    public synchronized void seekToBeginning(TopicPartition... partitions) {
-        ensureNotClosed();
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public synchronized void seekToEnd(TopicPartition... partitions) {
-        ensureNotClosed();
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public Map<MetricName, ? extends Metric> metrics() {
-        ensureNotClosed();
-        return Collections.emptyMap();
-    }
-
-    @Override
-    public synchronized List<PartitionInfo> partitionsFor(String topic) {
-        ensureNotClosed();
-        List<PartitionInfo> parts = this.partitions.get(topic);
-        if (parts == null)
-            return Collections.emptyList();
-        else
-            return parts;
-    }
-
-    public synchronized void updatePartitions(String topic, List<PartitionInfo> partitions) {
-        ensureNotClosed();
-        this.partitions.put(topic, partitions);
-    }
-
-    @Override
-    public synchronized void close() {
-        ensureNotClosed();
-        this.closed = true;
-    }
-
-    @Override
-    public void wakeup() {
-
-    }
-
-    private void ensureNotClosed() {
-        if (this.closed)
-            throw new IllegalStateException("This consumer has already been closed.");
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/NoOffsetForPartitionException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/NoOffsetForPartitionException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/NoOffsetForPartitionException.java
deleted file mode 100644
index 77be534..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/NoOffsetForPartitionException.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.kafka.copied.clients.consumer;
-
-import org.apache.kafka.copied.common.KafkaException;
-
-/**
- * Indicates that there is no stored offset and no defined offset reset policy
- */
-public class NoOffsetForPartitionException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public NoOffsetForPartitionException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/OffsetResetStrategy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/OffsetResetStrategy.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/OffsetResetStrategy.java
deleted file mode 100644
index 020844c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/OffsetResetStrategy.java
+++ /dev/null
@@ -1,17 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer;
-
-public enum OffsetResetStrategy {
-    LATEST, EARLIEST, NONE
-}


[03/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/MemoryRecords.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/MemoryRecords.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/MemoryRecords.java
deleted file mode 100644
index 94f55a6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/MemoryRecords.java
+++ /dev/null
@@ -1,271 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.record;
-
-import org.apache.kafka.copied.common.KafkaException;
-import org.apache.kafka.copied.common.utils.AbstractIterator;
-
-import java.io.DataInputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.GatheringByteChannel;
-import java.util.Iterator;
-
-/**
- * A {@link Records} implementation backed by a ByteBuffer.
- */
-public class MemoryRecords implements Records {
-
-    private final Compressor compressor;
-    private final int capacity;
-    private final int sizeLimit;
-    private ByteBuffer buffer;
-    private boolean writable;
-
-    // Construct a writable memory records
-    private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int sizeLimit) {
-        this.writable = writable;
-        this.capacity = buffer.capacity();
-        this.sizeLimit = sizeLimit;
-        if (this.writable) {
-            this.buffer = null;
-            this.compressor = new Compressor(buffer, type);
-        } else {
-            this.buffer = buffer;
-            this.compressor = null;
-        }
-    }
-
-    public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int capacity) {
-        return new MemoryRecords(buffer, type, true, capacity);
-    }
-
-    public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) {
-        return emptyRecords(buffer, type, buffer.capacity());
-    }
-
-    public static MemoryRecords readableRecords(ByteBuffer buffer) {
-        return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity());
-    }
-
-    /**
-     * Append the given record and offset to the buffer
-     */
-    public void append(long offset, Record record) {
-        if (!writable)
-            throw new IllegalStateException("Memory records is not writable");
-
-        int size = record.size();
-        compressor.putLong(offset);
-        compressor.putInt(size);
-        compressor.put(record.buffer());
-        compressor.recordWritten(size + LOG_OVERHEAD);
-        record.buffer().rewind();
-    }
-
-    /**
-     * Append a new record and offset to the buffer
-     */
-    public void append(long offset, byte[] key, byte[] value) {
-        if (!writable)
-            throw new IllegalStateException("Memory records is not writable");
-
-        int size = Record.recordSize(key, value);
-        compressor.putLong(offset);
-        compressor.putInt(size);
-        compressor.putRecord(key, value);
-        compressor.recordWritten(size + LOG_OVERHEAD);
-    }
-
-    /**
-     * Check if we have room for a new record containing the given key/value pair
-     * 
-     * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
-     * accurate if compression is really used. When this happens, the following append may cause dynamic buffer
-     * re-allocation in the underlying byte buffer stream.
-     * 
-     * Also note that besides the records' capacity, there is also a size limit for the batch. This size limit may be
-     * smaller than the capacity (e.g. when appending a single message whose size is larger than the batch size, the
-     * capacity will be the message size, but the size limit will still be the batch size), and when the records' size
-     * has exceed this limit we also mark this record as full.
-     */
-    public boolean hasRoomFor(byte[] key, byte[] value) {
-        return this.writable && this.capacity >= this.compressor.estimatedBytesWritten() + LOG_OVERHEAD +
-                                                 Record.recordSize(key, value) &&
-               this.sizeLimit >= this.compressor.estimatedBytesWritten();
-    }
-
-    public boolean isFull() {
-        return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten() ||
-               this.sizeLimit <= this.compressor.estimatedBytesWritten();
-    }
-
-    /**
-     * Close this batch for no more appends
-     */
-    public void close() {
-        if (writable) {
-            compressor.close();
-            writable = false;
-            buffer = compressor.buffer();
-        }
-    }
-
-    /** Write the records in this set to the given channel */
-    public int writeTo(GatheringByteChannel channel) throws IOException {
-        return channel.write(buffer);
-    }
-
-    /**
-     * The size of this record set
-     */
-    public int sizeInBytes() {
-        return compressor.buffer().position();
-    }
-
-    /**
-     * The compression rate of this record set
-     */
-    public double compressionRate() {
-        if (compressor == null)
-            return 1.0;
-        else
-            return compressor.compressionRate();
-    }
-
-    /**
-     * Return the capacity of the buffer
-     */
-    public int capacity() {
-        return this.capacity;
-    }
-
-    /**
-     * Get the byte buffer that backs this records instance
-     */
-    public ByteBuffer buffer() {
-        return buffer.duplicate();
-    }
-
-    /**
-     * Return a flipped duplicate of the closed buffer to reading records
-     */
-    public ByteBuffer flip() {
-        if (writable)
-            throw new IllegalStateException("The memory records need to be closed for write before rewinding for read");
-
-        return (ByteBuffer) buffer.flip();
-    }
-
-    @Override
-    public Iterator<LogEntry> iterator() {
-        ByteBuffer copy = this.buffer.duplicate();
-        return new RecordsIterator(copy, CompressionType.NONE, false);
-    }
-    
-    @Override
-    public String toString() {
-        Iterator<LogEntry> iter = iterator();
-        StringBuilder builder = new StringBuilder();
-        builder.append('[');
-        while (iter.hasNext()) {
-            LogEntry entry = iter.next();
-            builder.append('(');
-            builder.append("offset=");
-            builder.append(entry.offset());
-            builder.append(",");
-            builder.append("record=");
-            builder.append(entry.record());
-            builder.append(")");
-        }
-        builder.append(']');
-        return builder.toString();
-    }
-
-    public static class RecordsIterator extends AbstractIterator<LogEntry> {
-        private final ByteBuffer buffer;
-        private final DataInputStream stream;
-        private final CompressionType type;
-        private final boolean shallow;
-        private RecordsIterator innerIter;
-
-        public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) {
-            this.type = type;
-            this.buffer = buffer;
-            this.shallow = shallow;
-            this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type);
-        }
-
-        /*
-         * Read the next record from the buffer.
-         * 
-         * Note that in the compressed message set, each message value size is set as the size of the un-compressed
-         * version of the message value, so when we do de-compression allocating an array of the specified size for
-         * reading compressed value data is sufficient.
-         */
-        @Override
-        protected LogEntry makeNext() {
-            if (innerDone()) {
-                try {
-                    // read the offset
-                    long offset = stream.readLong();
-                    // read record size
-                    int size = stream.readInt();
-                    if (size < 0)
-                        throw new IllegalStateException("Record with size " + size);
-                    // read the record, if compression is used we cannot depend on size
-                    // and hence has to do extra copy
-                    ByteBuffer rec;
-                    if (type == CompressionType.NONE) {
-                        rec = buffer.slice();
-                        int newPos = buffer.position() + size;
-                        if (newPos > buffer.limit())
-                            return allDone();
-                        buffer.position(newPos);
-                        rec.limit(size);
-                    } else {
-                        byte[] recordBuffer = new byte[size];
-                        stream.readFully(recordBuffer, 0, size);
-                        rec = ByteBuffer.wrap(recordBuffer);
-                    }
-                    LogEntry entry = new LogEntry(offset, new Record(rec));
-
-                    // decide whether to go shallow or deep iteration if it is compressed
-                    CompressionType compression = entry.record().compressionType();
-                    if (compression == CompressionType.NONE || shallow) {
-                        return entry;
-                    } else {
-                        // init the inner iterator with the value payload of the message,
-                        // which will de-compress the payload to a set of messages;
-                        // since we assume nested compression is not allowed, the deep iterator
-                        // would not try to further decompress underlying messages
-                        ByteBuffer value = entry.record().value();
-                        innerIter = new RecordsIterator(value, compression, true);
-                        return innerIter.next();
-                    }
-                } catch (EOFException e) {
-                    return allDone();
-                } catch (IOException e) {
-                    throw new KafkaException(e);
-                }
-            } else {
-                return innerIter.next();
-            }
-        }
-
-        private boolean innerDone() {
-            return innerIter == null || !innerIter.hasNext();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Record.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Record.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Record.java
deleted file mode 100644
index 54b4084..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Record.java
+++ /dev/null
@@ -1,344 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.record;
-
-import org.apache.kafka.copied.common.utils.Crc32;
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.nio.ByteBuffer;
-
-
-/**
- * A record: a serialized key and value along with the associated CRC and other fields
- */
-public final class Record {
-
-    /**
-     * The current offset and size for all the fixed-length fields
-     */
-    public static final int CRC_OFFSET = 0;
-    public static final int CRC_LENGTH = 4;
-    public static final int MAGIC_OFFSET = CRC_OFFSET + CRC_LENGTH;
-    public static final int MAGIC_LENGTH = 1;
-    public static final int ATTRIBUTES_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
-    public static final int ATTRIBUTE_LENGTH = 1;
-    public static final int KEY_SIZE_OFFSET = ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH;
-    public static final int KEY_SIZE_LENGTH = 4;
-    public static final int KEY_OFFSET = KEY_SIZE_OFFSET + KEY_SIZE_LENGTH;
-    public static final int VALUE_SIZE_LENGTH = 4;
-
-    /**
-     * The size for the record header
-     */
-    public static final int HEADER_SIZE = CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH;
-
-    /**
-     * The amount of overhead bytes in a record
-     */
-    public static final int RECORD_OVERHEAD = HEADER_SIZE + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
-
-    /**
-     * The current "magic" value
-     */
-    public static final byte CURRENT_MAGIC_VALUE = 0;
-
-    /**
-     * Specifies the mask for the compression code. 3 bits to hold the compression codec. 0 is reserved to indicate no
-     * compression
-     */
-    public static final int COMPRESSION_CODEC_MASK = 0x07;
-
-    /**
-     * Compression code for uncompressed records
-     */
-    public static final int NO_COMPRESSION = 0;
-
-    private final ByteBuffer buffer;
-
-    public Record(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    /**
-     * A constructor to create a LogRecord. If the record's compression type is not none, then
-     * its value payload should be already compressed with the specified type; the constructor
-     * would always write the value payload as is and will not do the compression itself.
-     * 
-     * @param key The key of the record (null, if none)
-     * @param value The record value
-     * @param type The compression type used on the contents of the record (if any)
-     * @param valueOffset The offset into the payload array used to extract payload
-     * @param valueSize The size of the payload to use
-     */
-    public Record(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length,
-                value == null ? 0 : valueSize >= 0 ? valueSize : value.length - valueOffset)));
-        write(this.buffer, key, value, type, valueOffset, valueSize);
-        this.buffer.rewind();
-    }
-
-    public Record(byte[] key, byte[] value, CompressionType type) {
-        this(key, value, type, 0, -1);
-    }
-
-    public Record(byte[] value, CompressionType type) {
-        this(null, value, type);
-    }
-
-    public Record(byte[] key, byte[] value) {
-        this(key, value, CompressionType.NONE);
-    }
-
-    public Record(byte[] value) {
-        this(null, value, CompressionType.NONE);
-    }
-
-    // Write a record to the buffer, if the record's compression type is none, then
-    // its value payload should be already compressed with the specified type
-    public static void write(ByteBuffer buffer, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        // construct the compressor with compression type none since this function will not do any
-        //compression according to the input type, it will just write the record's payload as is
-        Compressor compressor = new Compressor(buffer, CompressionType.NONE, buffer.capacity());
-        compressor.putRecord(key, value, type, valueOffset, valueSize);
-    }
-
-    public static void write(Compressor compressor, long crc, byte attributes, byte[] key, byte[] value, int valueOffset, int valueSize) {
-        // write crc
-        compressor.putInt((int) (crc & 0xffffffffL));
-        // write magic value
-        compressor.putByte(CURRENT_MAGIC_VALUE);
-        // write attributes
-        compressor.putByte(attributes);
-        // write the key
-        if (key == null) {
-            compressor.putInt(-1);
-        } else {
-            compressor.putInt(key.length);
-            compressor.put(key, 0, key.length);
-        }
-        // write the value
-        if (value == null) {
-            compressor.putInt(-1);
-        } else {
-            int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
-            compressor.putInt(size);
-            compressor.put(value, valueOffset, size);
-        }
-    }
-
-    public static int recordSize(byte[] key, byte[] value) {
-        return recordSize(key == null ? 0 : key.length, value == null ? 0 : value.length);
-    }
-
-    public static int recordSize(int keySize, int valueSize) {
-        return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
-    }
-
-    public ByteBuffer buffer() {
-        return this.buffer;
-    }
-
-    public static byte computeAttributes(CompressionType type) {
-        byte attributes = 0;
-        if (type.id > 0)
-            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
-        return attributes;
-    }
-
-    /**
-     * Compute the checksum of the record from the record contents
-     */
-    public static long computeChecksum(ByteBuffer buffer, int position, int size) {
-        Crc32 crc = new Crc32();
-        crc.update(buffer.array(), buffer.arrayOffset() + position, size);
-        return crc.getValue();
-    }
-
-    /**
-     * Compute the checksum of the record from the attributes, key and value payloads
-     */
-    public static long computeChecksum(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        Crc32 crc = new Crc32();
-        crc.update(CURRENT_MAGIC_VALUE);
-        byte attributes = 0;
-        if (type.id > 0)
-            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
-        crc.update(attributes);
-        // update for the key
-        if (key == null) {
-            crc.updateInt(-1);
-        } else {
-            crc.updateInt(key.length);
-            crc.update(key, 0, key.length);
-        }
-        // update for the value
-        if (value == null) {
-            crc.updateInt(-1);
-        } else {
-            int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
-            crc.updateInt(size);
-            crc.update(value, valueOffset, size);
-        }
-        return crc.getValue();
-    }
-
-
-    /**
-     * Compute the checksum of the record from the record contents
-     */
-    public long computeChecksum() {
-        return computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
-    }
-
-    /**
-     * Retrieve the previously computed CRC for this record
-     */
-    public long checksum() {
-        return Utils.readUnsignedInt(buffer, CRC_OFFSET);
-    }
-
-    /**
-     * Returns true if the crc stored with the record matches the crc computed off the record contents
-     */
-    public boolean isValid() {
-        return checksum() == computeChecksum();
-    }
-
-    /**
-     * Throw an InvalidRecordException if isValid is false for this record
-     */
-    public void ensureValid() {
-        if (!isValid())
-            throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
-                                             + ", computed crc = "
-                                             + computeChecksum()
-                                             + ")");
-    }
-
-    /**
-     * The complete serialized size of this record in bytes (including crc, header attributes, etc)
-     */
-    public int size() {
-        return buffer.limit();
-    }
-
-    /**
-     * The length of the key in bytes
-     */
-    public int keySize() {
-        return buffer.getInt(KEY_SIZE_OFFSET);
-    }
-
-    /**
-     * Does the record have a key?
-     */
-    public boolean hasKey() {
-        return keySize() >= 0;
-    }
-
-    /**
-     * The position where the value size is stored
-     */
-    private int valueSizeOffset() {
-        return KEY_OFFSET + Math.max(0, keySize());
-    }
-
-    /**
-     * The length of the value in bytes
-     */
-    public int valueSize() {
-        return buffer.getInt(valueSizeOffset());
-    }
-
-    /**
-     * The magic version of this record
-     */
-    public byte magic() {
-        return buffer.get(MAGIC_OFFSET);
-    }
-
-    /**
-     * The attributes stored with this record
-     */
-    public byte attributes() {
-        return buffer.get(ATTRIBUTES_OFFSET);
-    }
-
-    /**
-     * The compression type used with this record
-     */
-    public CompressionType compressionType() {
-        return CompressionType.forId(buffer.get(ATTRIBUTES_OFFSET) & COMPRESSION_CODEC_MASK);
-    }
-
-    /**
-     * A ByteBuffer containing the value of this record
-     */
-    public ByteBuffer value() {
-        return sliceDelimited(valueSizeOffset());
-    }
-
-    /**
-     * A ByteBuffer containing the message key
-     */
-    public ByteBuffer key() {
-        return sliceDelimited(KEY_SIZE_OFFSET);
-    }
-
-    /**
-     * Read a size-delimited byte buffer starting at the given offset
-     */
-    private ByteBuffer sliceDelimited(int start) {
-        int size = buffer.getInt(start);
-        if (size < 0) {
-            return null;
-        } else {
-            ByteBuffer b = buffer.duplicate();
-            b.position(start + 4);
-            b = b.slice();
-            b.limit(size);
-            b.rewind();
-            return b;
-        }
-    }
-
-    public String toString() {
-        return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, key = %d bytes, value = %d bytes)",
-                magic(),
-                attributes(),
-                compressionType(),
-                checksum(),
-                key() == null ? 0 : key().limit(),
-                value() == null ? 0 : value().limit());
-    }
-
-    public boolean equals(Object other) {
-        if (this == other)
-            return true;
-        if (other == null)
-            return false;
-        if (!other.getClass().equals(Record.class))
-            return false;
-        Record record = (Record) other;
-        return this.buffer.equals(record.buffer);
-    }
-
-    public int hashCode() {
-        return buffer.hashCode();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Records.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Records.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Records.java
deleted file mode 100644
index 06f6668..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Records.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.kafka.copied.common.record;
-
-import java.io.IOException;
-import java.nio.channels.GatheringByteChannel;
-
-/**
- * A binary format which consists of a 4 byte size, an 8 byte offset, and the record bytes. See {@link MemoryRecords}
- * for the in-memory representation.
- */
-public interface Records extends Iterable<LogEntry> {
-
-    int SIZE_LENGTH = 4;
-    int OFFSET_LENGTH = 8;
-    int LOG_OVERHEAD = SIZE_LENGTH + OFFSET_LENGTH;
-
-    /**
-     * Write these records to the given channel
-     * @param channel The channel to write to
-     * @return The number of bytes written
-     * @throws IOException If the write fails.
-     */
-    public int writeTo(GatheringByteChannel channel) throws IOException;
-
-    /**
-     * The size of these records in bytes
-     */
-    public int sizeInBytes();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/AbstractRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/AbstractRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/AbstractRequest.java
deleted file mode 100644
index 53972c9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/AbstractRequest.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-public abstract class AbstractRequest extends AbstractRequestResponse {
-
-    public AbstractRequest(Struct struct) {
-        super(struct);
-    }
-
-    /**
-     * Get an error response for a request for a given api version
-     */
-    public abstract AbstractRequestResponse getErrorResponse(int versionId, Throwable e);
-
-    /**
-     * Factory method for getting a request object based on ApiKey ID and a buffer
-     */
-    public static AbstractRequest getRequest(int requestId, int versionId, ByteBuffer buffer) {
-        switch (ApiKeys.forId(requestId)) {
-            case PRODUCE:
-                return ProduceRequest.parse(buffer, versionId);
-            case FETCH:
-                return FetchRequest.parse(buffer, versionId);
-            case LIST_OFFSETS:
-                return ListOffsetRequest.parse(buffer, versionId);
-            case METADATA:
-                return MetadataRequest.parse(buffer, versionId);
-            case OFFSET_COMMIT:
-                return OffsetCommitRequest.parse(buffer, versionId);
-            case OFFSET_FETCH:
-                return OffsetFetchRequest.parse(buffer, versionId);
-            case CONSUMER_METADATA:
-                return ConsumerMetadataRequest.parse(buffer, versionId);
-            case JOIN_GROUP:
-                return JoinGroupRequest.parse(buffer, versionId);
-            case HEARTBEAT:
-                return HeartbeatRequest.parse(buffer, versionId);
-            default:
-                return null;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/AbstractRequestResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/AbstractRequestResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/AbstractRequestResponse.java
deleted file mode 100644
index b070fec..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/AbstractRequestResponse.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-public abstract class AbstractRequestResponse {
-    protected final Struct struct;
-
-
-    public AbstractRequestResponse(Struct struct) {
-        this.struct = struct;
-    }
-
-    public Struct toStruct() {
-        return struct;
-    }
-
-    /**
-     * Get the serialized size of this object
-     */
-    public int sizeOf() {
-        return struct.sizeOf();
-    }
-
-    /**
-     * Write this object to a buffer
-     */
-    public void writeTo(ByteBuffer buffer) {
-        struct.writeTo(buffer);
-    }
-
-    @Override
-    public String toString() {
-        return struct.toString();
-    }
-
-    @Override
-    public int hashCode() {
-        return struct.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        AbstractRequestResponse other = (AbstractRequestResponse) obj;
-        return struct.equals(other.struct);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ConsumerMetadataRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ConsumerMetadataRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ConsumerMetadataRequest.java
deleted file mode 100644
index c5d65b2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ConsumerMetadataRequest.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-public class ConsumerMetadataRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-
-    private final String groupId;
-
-    public ConsumerMetadataRequest(String groupId) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        this.groupId = groupId;
-    }
-
-    public ConsumerMetadataRequest(Struct struct) {
-        super(struct);
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        switch (versionId) {
-            case 0:
-                return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode());
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CONSUMER_METADATA.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public static ConsumerMetadataRequest parse(ByteBuffer buffer, int versionId) {
-        return new ConsumerMetadataRequest(ProtoUtils.parseRequest(ApiKeys.CONSUMER_METADATA.id, versionId, buffer));
-    }
-
-    public static ConsumerMetadataRequest parse(ByteBuffer buffer) {
-        return new ConsumerMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ConsumerMetadataResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ConsumerMetadataResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ConsumerMetadataResponse.java
deleted file mode 100644
index 68fd4cb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ConsumerMetadataResponse.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-public class ConsumerMetadataResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id);
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-    private static final String COORDINATOR_KEY_NAME = "coordinator";
-
-    // coordinator level field names
-    private static final String NODE_ID_KEY_NAME = "node_id";
-    private static final String HOST_KEY_NAME = "host";
-    private static final String PORT_KEY_NAME = "port";
-
-    private final short errorCode;
-    private final Node node;
-
-    public ConsumerMetadataResponse(short errorCode, Node node) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(ERROR_CODE_KEY_NAME, errorCode);
-        Struct coordinator = struct.instance(COORDINATOR_KEY_NAME);
-        coordinator.set(NODE_ID_KEY_NAME, node.id());
-        coordinator.set(HOST_KEY_NAME, node.host());
-        coordinator.set(PORT_KEY_NAME, node.port());
-        struct.set(COORDINATOR_KEY_NAME, coordinator);
-        this.errorCode = errorCode;
-        this.node = node;
-    }
-
-    public ConsumerMetadataResponse(Struct struct) {
-        super(struct);
-        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
-        Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME);
-        int nodeId = broker.getInt(NODE_ID_KEY_NAME);
-        String host = broker.getString(HOST_KEY_NAME);
-        int port = broker.getInt(PORT_KEY_NAME);
-        node = new Node(nodeId, host, port);
-    }
-
-    public short errorCode() {
-        return errorCode;
-    }
-
-    public Node node() {
-        return node;
-    }
-
-    public static ConsumerMetadataResponse parse(ByteBuffer buffer) {
-        return new ConsumerMetadataResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/FetchRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/FetchRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/FetchRequest.java
deleted file mode 100644
index 78069ae..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/FetchRequest.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class FetchRequest extends AbstractRequest {
-    
-    public static final int CONSUMER_REPLICA_ID = -1;
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id);
-    private static final String REPLICA_ID_KEY_NAME = "replica_id";
-    private static final String MAX_WAIT_KEY_NAME = "max_wait_time";
-    private static final String MIN_BYTES_KEY_NAME = "min_bytes";
-    private static final String TOPICS_KEY_NAME = "topics";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset";
-    private static final String MAX_BYTES_KEY_NAME = "max_bytes";
-
-    private final int replicaId;
-    private final int maxWait;
-    private final int minBytes;
-    private final Map<TopicPartition, PartitionData> fetchData;
-
-    public static final class PartitionData {
-        public final long offset;
-        public final int maxBytes;
-
-        public PartitionData(long offset, int maxBytes) {
-            this.offset = offset;
-            this.maxBytes = maxBytes;
-        }
-    }
-
-    /**
-     * Create a non-replica fetch request
-     */
-    public FetchRequest(int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
-        this(CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData);
-    }
-
-    /**
-     * Create a replica fetch request
-     */
-    public FetchRequest(int replicaId, int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(fetchData);
-
-        struct.set(REPLICA_ID_KEY_NAME, replicaId);
-        struct.set(MAX_WAIT_KEY_NAME, maxWait);
-        struct.set(MIN_BYTES_KEY_NAME, minBytes);
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry : topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset);
-                partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
-        this.replicaId = replicaId;
-        this.maxWait = maxWait;
-        this.minBytes = minBytes;
-        this.fetchData = fetchData;
-    }
-
-    public FetchRequest(Struct struct) {
-        super(struct);
-        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
-        maxWait = struct.getInt(MAX_WAIT_KEY_NAME);
-        minBytes = struct.getInt(MIN_BYTES_KEY_NAME);
-        fetchData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                long offset = partitionResponse.getLong(FETCH_OFFSET_KEY_NAME);
-                int maxBytes = partitionResponse.getInt(MAX_BYTES_KEY_NAME);
-                PartitionData partitionData = new PartitionData(offset, maxBytes);
-                fetchData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<TopicPartition, FetchResponse.PartitionData> responseData = new HashMap<TopicPartition, FetchResponse.PartitionData>();
-
-        for (Map.Entry<TopicPartition, PartitionData> entry: fetchData.entrySet()) {
-            FetchResponse.PartitionData partitionResponse = new FetchResponse.PartitionData(Errors.forException(e).code(),
-                    FetchResponse.INVALID_HIGHWATERMARK,
-                    FetchResponse.EMPTY_RECORD_SET);
-            responseData.put(entry.getKey(), partitionResponse);
-        }
-
-        switch (versionId) {
-            case 0:
-                return new FetchResponse(responseData);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.FETCH.id)));
-        }
-    }
-
-    public int replicaId() {
-        return replicaId;
-    }
-
-    public int maxWait() {
-        return maxWait;
-    }
-
-    public int minBytes() {
-        return minBytes;
-    }
-
-    public Map<TopicPartition, PartitionData> fetchData() {
-        return fetchData;
-    }
-
-    public static FetchRequest parse(ByteBuffer buffer, int versionId) {
-        return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer));
-    }
-
-    public static FetchRequest parse(ByteBuffer buffer) {
-        return new FetchRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/FetchResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/FetchResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/FetchResponse.java
deleted file mode 100644
index fc2c44e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/FetchResponse.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class FetchResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     *  OFFSET_OUT_OF_RANGE (1)
-     *  UNKNOWN_TOPIC_OR_PARTITION (3)
-     *  NOT_LEADER_FOR_PARTITION (6)
-     *  REPLICA_NOT_AVAILABLE (9)
-     *  UNKNOWN (-1)
-     */
-
-    private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark";
-    private static final String RECORD_SET_KEY_NAME = "record_set";
-
-    public static final long INVALID_HIGHWATERMARK = -1L;
-    public static final ByteBuffer EMPTY_RECORD_SET = ByteBuffer.allocate(0);
-
-    private final Map<TopicPartition, PartitionData> responseData;
-
-    public static final class PartitionData {
-        public final short errorCode;
-        public final long highWatermark;
-        public final ByteBuffer recordSet;
-
-        public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) {
-            this.errorCode = errorCode;
-            this.highWatermark = highWatermark;
-            this.recordSet = recordSet;
-        }
-    }
-
-    public FetchResponse(Map<TopicPartition, PartitionData> responseData) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
-                partitionData.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark);
-                partitionData.set(RECORD_SET_KEY_NAME, fetchPartitionData.recordSet);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
-        this.responseData = responseData;
-    }
-
-    public FetchResponse(Struct struct) {
-        super(struct);
-        responseData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
-                long highWatermark = partitionResponse.getLong(HIGH_WATERMARK_KEY_NAME);
-                ByteBuffer recordSet = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
-                PartitionData partitionData = new PartitionData(errorCode, highWatermark, recordSet);
-                responseData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    public Map<TopicPartition, PartitionData> responseData() {
-        return responseData;
-    }
-
-    public static FetchResponse parse(ByteBuffer buffer) {
-        return new FetchResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/HeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/HeartbeatRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/HeartbeatRequest.java
deleted file mode 100644
index 7cb7699..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/HeartbeatRequest.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-public class HeartbeatRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-    private static final String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id";
-    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
-
-    private final String groupId;
-    private final int groupGenerationId;
-    private final String consumerId;
-
-    public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        this.groupId = groupId;
-        this.groupGenerationId = groupGenerationId;
-        this.consumerId = consumerId;
-    }
-
-    public HeartbeatRequest(Struct struct) {
-        super(struct);
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-        groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME);
-        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        switch (versionId) {
-            case 0:
-                return new HeartbeatResponse(Errors.forException(e).code());
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.HEARTBEAT.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public int groupGenerationId() {
-        return groupGenerationId;
-    }
-
-    public String consumerId() {
-        return consumerId;
-    }
-
-    public static HeartbeatRequest parse(ByteBuffer buffer, int versionId) {
-        return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer));
-    }
-
-    public static HeartbeatRequest parse(ByteBuffer buffer) {
-        return new HeartbeatRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/HeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/HeartbeatResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/HeartbeatResponse.java
deleted file mode 100644
index 12c5ac8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/HeartbeatResponse.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-public class HeartbeatResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id);
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR_FOR_CONSUMER (16)
-     * ILLEGAL_GENERATION (22)
-     * UNKNOWN_CONSUMER_ID (25)
-     */
-
-    private final short errorCode;
-    public HeartbeatResponse(short errorCode) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(ERROR_CODE_KEY_NAME, errorCode);
-        this.errorCode = errorCode;
-    }
-
-    public HeartbeatResponse(Struct struct) {
-        super(struct);
-        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
-    }
-
-    public short errorCode() {
-        return errorCode;
-    }
-
-    public static HeartbeatResponse parse(ByteBuffer buffer) {
-        return new HeartbeatResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/JoinGroupRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/JoinGroupRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/JoinGroupRequest.java
deleted file mode 100644
index 011bb9d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/JoinGroupRequest.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class JoinGroupRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-    private static final String SESSION_TIMEOUT_KEY_NAME = "session_timeout";
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
-    private static final String STRATEGY_KEY_NAME = "partition_assignment_strategy";
-
-    public static final String UNKNOWN_CONSUMER_ID = "";
-
-    private final String groupId;
-    private final int sessionTimeout;
-    private final List<String> topics;
-    private final String consumerId;
-    private final String strategy;
-
-    public JoinGroupRequest(String groupId, int sessionTimeout, List<String> topics, String consumerId, String strategy) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout);
-        struct.set(TOPICS_KEY_NAME, topics.toArray());
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        struct.set(STRATEGY_KEY_NAME, strategy);
-        this.groupId = groupId;
-        this.sessionTimeout = sessionTimeout;
-        this.topics = topics;
-        this.consumerId = consumerId;
-        this.strategy = strategy;
-    }
-
-    public JoinGroupRequest(Struct struct) {
-        super(struct);
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-        sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME);
-        Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME);
-        topics = new ArrayList<String>();
-        for (Object topic: topicsArray)
-            topics.add((String) topic);
-        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
-        strategy = struct.getString(STRATEGY_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        switch (versionId) {
-            case 0:
-                return new JoinGroupResponse(
-                        Errors.forException(e).code(),
-                        JoinGroupResponse.UNKNOWN_GENERATION_ID,
-                        JoinGroupResponse.UNKNOWN_CONSUMER_ID,
-                        Collections.<TopicPartition>emptyList());
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public int sessionTimeout() {
-        return sessionTimeout;
-    }
-
-    public List<String> topics() {
-        return topics;
-    }
-
-    public String consumerId() {
-        return consumerId;
-    }
-
-    public String strategy() {
-        return strategy;
-    }
-
-    public static JoinGroupRequest parse(ByteBuffer buffer, int versionId) {
-        return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, versionId, buffer));
-    }
-
-    public static JoinGroupRequest parse(ByteBuffer buffer) {
-        return new JoinGroupRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/JoinGroupResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/JoinGroupResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/JoinGroupResponse.java
deleted file mode 100644
index 5855e94..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/JoinGroupResponse.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-public class JoinGroupResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id);
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR_FOR_CONSUMER (16)
-     * INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY (23)
-     * UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY (24)
-     * UNKNOWN_CONSUMER_ID (25)
-     * INVALID_SESSION_TIMEOUT (26)
-     */
-
-    private static final String GENERATION_ID_KEY_NAME = "group_generation_id";
-    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
-    private static final String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions";
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    public static final int UNKNOWN_GENERATION_ID = -1;
-    public static final String UNKNOWN_CONSUMER_ID = "";
-
-    private final short errorCode;
-    private final int generationId;
-    private final String consumerId;
-    private final List<TopicPartition> assignedPartitions;
-
-    public JoinGroupResponse(short errorCode, int generationId, String consumerId, List<TopicPartition> assignedPartitions) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        Map<String, List<Integer>> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions);
-
-        struct.set(ERROR_CODE_KEY_NAME, errorCode);
-        struct.set(GENERATION_ID_KEY_NAME, generationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, List<Integer>> entries: partitionsByTopic.entrySet()) {
-            Struct topicData = struct.instance(ASSIGNED_PARTITIONS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
-            topicData.set(PARTITIONS_KEY_NAME, entries.getValue().toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(ASSIGNED_PARTITIONS_KEY_NAME, topicArray.toArray());
-
-        this.errorCode = errorCode;
-        this.generationId = generationId;
-        this.consumerId = consumerId;
-        this.assignedPartitions = assignedPartitions;
-    }
-
-    public JoinGroupResponse(Struct struct) {
-        super(struct);
-        assignedPartitions = new ArrayList<TopicPartition>();
-        for (Object topicDataObj : struct.getArray(ASSIGNED_PARTITIONS_KEY_NAME)) {
-            Struct topicData = (Struct) topicDataObj;
-            String topic = topicData.getString(TOPIC_KEY_NAME);
-            for (Object partitionObj : topicData.getArray(PARTITIONS_KEY_NAME))
-                assignedPartitions.add(new TopicPartition(topic, (Integer) partitionObj));
-        }
-        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
-        generationId = struct.getInt(GENERATION_ID_KEY_NAME);
-        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
-    }
-
-    public short errorCode() {
-        return errorCode;
-    }
-
-    public int generationId() {
-        return generationId;
-    }
-
-    public String consumerId() {
-        return consumerId;
-    }
-
-    public List<TopicPartition> assignedPartitions() {
-        return assignedPartitions;
-    }
-
-    public static JoinGroupResponse parse(ByteBuffer buffer) {
-        return new JoinGroupResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ListOffsetRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ListOffsetRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ListOffsetRequest.java
deleted file mode 100644
index 299de92..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ListOffsetRequest.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class ListOffsetRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id);
-    private static final String REPLICA_ID_KEY_NAME = "replica_id";
-    private static final String TOPICS_KEY_NAME = "topics";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String TIMESTAMP_KEY_NAME = "timestamp";
-    private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets";
-
-    private final int replicaId;
-    private final Map<TopicPartition, PartitionData> offsetData;
-
-    public static final class PartitionData {
-        public final long timestamp;
-        public final int maxNumOffsets;
-
-        public PartitionData(long timestamp, int maxNumOffsets) {
-            this.timestamp = timestamp;
-            this.maxNumOffsets = maxNumOffsets;
-        }
-    }
-    
-    public ListOffsetRequest(Map<TopicPartition, PartitionData> offsetData) {
-        this(-1, offsetData);
-    }
-
-    public ListOffsetRequest(int replicaId, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
-
-        struct.set(REPLICA_ID_KEY_NAME, replicaId);
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData offsetPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp);
-                partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
-        this.replicaId = replicaId;
-        this.offsetData = offsetData;
-    }
-
-    public ListOffsetRequest(Struct struct) {
-        super(struct);
-        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
-        offsetData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
-                int maxNumOffsets = partitionResponse.getInt(MAX_NUM_OFFSETS_KEY_NAME);
-                PartitionData partitionData = new PartitionData(timestamp, maxNumOffsets);
-                offsetData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<TopicPartition, ListOffsetResponse.PartitionData> responseData = new HashMap<TopicPartition, ListOffsetResponse.PartitionData>();
-
-        for (Map.Entry<TopicPartition, PartitionData> entry: offsetData.entrySet()) {
-            ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(Errors.forException(e).code(), new ArrayList<Long>());
-            responseData.put(entry.getKey(), partitionResponse);
-        }
-
-        switch (versionId) {
-            case 0:
-                return new ListOffsetResponse(responseData);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id)));
-        }
-    }
-
-    public int replicaId() {
-        return replicaId;
-    }
-
-    public Map<TopicPartition, PartitionData> offsetData() {
-        return offsetData;
-    }
-
-    public static ListOffsetRequest parse(ByteBuffer buffer, int versionId) {
-        return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer));
-    }
-
-    public static ListOffsetRequest parse(ByteBuffer buffer) {
-        return new ListOffsetRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ListOffsetResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ListOffsetResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ListOffsetResponse.java
deleted file mode 100644
index d7de2df..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ListOffsetResponse.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class ListOffsetResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     *  UNKNOWN_TOPIC_OR_PARTITION (3)
-     *  NOT_LEADER_FOR_PARTITION (6)
-     *  UNKNOWN (-1)
-     */
-
-    private static final String OFFSETS_KEY_NAME = "offsets";
-
-    private final Map<TopicPartition, PartitionData> responseData;
-
-    public static final class PartitionData {
-        public final short errorCode;
-        public final List<Long> offsets;
-
-        public PartitionData(short errorCode, List<Long> offsets) {
-            this.errorCode = errorCode;
-            this.offsets = offsets;
-        }
-    }
-
-    public ListOffsetResponse(Map<TopicPartition, PartitionData> responseData) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData offsetPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.errorCode);
-                partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray());
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
-        this.responseData = responseData;
-    }
-
-    public ListOffsetResponse(Struct struct) {
-        super(struct);
-        responseData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
-                Object[] offsets = partitionResponse.getArray(OFFSETS_KEY_NAME);
-                List<Long> offsetsList = new ArrayList<Long>();
-                for (Object offset: offsets)
-                    offsetsList.add((Long) offset);
-                PartitionData partitionData = new PartitionData(errorCode, offsetsList);
-                responseData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    public Map<TopicPartition, PartitionData> responseData() {
-        return responseData;
-    }
-
-    public static ListOffsetResponse parse(ByteBuffer buffer) {
-        return new ListOffsetResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}


[28/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Refactor, cleanup, and fix kafka consumers

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java
index 1f244c1..9638b84 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.streaming.connectors.internals;
 
-import com.google.common.base.Preconditions;
 import kafka.api.FetchRequestBuilder;
 import kafka.api.OffsetRequest;
 import kafka.api.PartitionOffsetRequestInfo;
@@ -27,345 +27,576 @@ import kafka.javaapi.OffsetResponse;
 import kafka.javaapi.consumer.SimpleConsumer;
 import kafka.javaapi.message.ByteBufferMessageSet;
 import kafka.message.MessageAndOffset;
-import org.apache.flink.api.java.tuple.Tuple2;
+
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.FlinkKafkaConsumer;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.util.StringUtils;
 import org.apache.flink.kafka_backport.common.Node;
 import org.apache.flink.kafka_backport.common.PartitionInfo;
 import org.apache.flink.kafka_backport.common.TopicPartition;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.google.common.base.Preconditions.checkNotNull;
 
+/**
+ * This fetcher uses Kafka's low-level API to pull data from a specific
+ * set of partitions and offsets for a certain topic.
+ * 
+ * <p>This code is in parts based on the tutorial code for the low-level Kafka consumer.</p>
+ */
 public class LegacyFetcher implements Fetcher {
-	public static Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumerBase.class);
+	
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer.class);
 
+	/** The topic from which this fetcher pulls data */
 	private final String topic;
+	
+	/** The properties that configure the Kafka connection */
+	private final Properties config;
+	
+	/** The task name, to give more readable names to the spawned threads */
+	private final String taskName;
+	
+	/** The first error that occurred in a connection thread */
+	private final AtomicReference<Throwable> error;
+
+	/** The partitions that the fetcher should read, with their starting offsets */
 	private Map<TopicPartition, Long> partitionsToRead;
-	private boolean running = true;
-	private Properties config;
-
-	public final static String QUEUE_SIZE_KEY = "flink.kafka.consumer.queue.size";
-	public final static String DEFAULT_QUEUE_SIZE = "10000";
+	
+	/** Reference the the thread that executed the run() method. */
+	private volatile Thread mainThread;
+	
+	/** Flag to shot the fetcher down */
+	private volatile boolean running = true;
 
 
 	public LegacyFetcher(String topic, Properties props) {
-		config = props;
-		this.topic = topic;
+		this(topic, props, "");
+	}
+	
+	public LegacyFetcher(String topic, Properties props, String taskName) {
+		this.config = checkNotNull(props, "The config properties cannot be null");
+		this.topic = checkNotNull(topic, "The topic cannot be null");
+		this.taskName = taskName;
+		this.error = new AtomicReference<>();
 	}
 
+	// ------------------------------------------------------------------------
+	//  Fetcher methods
+	// ------------------------------------------------------------------------
+	
 	@Override
-	public void partitionsToRead(List<TopicPartition> partitions) {
+	public void setPartitionsToRead(List<TopicPartition> partitions) {
 		partitionsToRead = new HashMap<TopicPartition, Long>(partitions.size());
-		for(TopicPartition tp: partitions) {
-			partitionsToRead.put(tp, FlinkKafkaConsumerBase.OFFSET_NOT_SET);
+		for (TopicPartition tp: partitions) {
+			partitionsToRead.put(tp, FlinkKafkaConsumer.OFFSET_NOT_SET);
 		}
 	}
 
 	@Override
 	public void seek(TopicPartition topicPartition, long offsetToRead) {
-		if(partitionsToRead == null) {
+		if (partitionsToRead == null) {
 			throw new IllegalArgumentException("No partitions to read set");
 		}
-		if(!partitionsToRead.containsKey(topicPartition)) {
-			throw new IllegalArgumentException("Can not set offset on a partition ("+topicPartition+") we are not going to read. " +
-					"Partitions to read "+partitionsToRead);
+		if (!partitionsToRead.containsKey(topicPartition)) {
+			throw new IllegalArgumentException("Can not set offset on a partition (" + topicPartition
+					+ ") we are not going to read. Partitions to read " + partitionsToRead);
 		}
 		partitionsToRead.put(topicPartition, offsetToRead);
 	}
-
+	
 	@Override
 	public void close() {
-		running = false;
+		// flag needs to be check by the run() method that creates the spawned threads
+		this.running = false;
+		
+		// all other cleanup is made by the run method itself
 	}
 
 	@Override
-	public <T> void run(SourceFunction.SourceContext<T> sourceContext, DeserializationSchema<T> valueDeserializer, long[] lastOffsets) {
-		if(partitionsToRead == null || partitionsToRead.size() == 0) {
+	public <T> void run(SourceFunction.SourceContext<T> sourceContext, 
+						DeserializationSchema<T> valueDeserializer,
+						long[] lastOffsets) throws Exception {
+		
+		if (partitionsToRead == null || partitionsToRead.size() == 0) {
 			throw new IllegalArgumentException("No partitions set");
 		}
+		
+		// NOTE: This method is needs to always release all resources it acquires
+		
+		this.mainThread = Thread.currentThread();
 
-		LOG.info("Reading from partitions "+partitionsToRead+" using the legacy fetcher");
+		LOG.info("Reading from partitions " + partitionsToRead + " using the legacy fetcher");
+		
 		// get lead broker for each partition
-		List<PartitionInfo> allPartitionsInTopic = FlinkKafkaConsumerBase.getPartitionsForTopic(topic, config);
-
+		
+		// NOTE: The kafka client apparently locks itself in an infinite loop sometimes
+		// when it is interrupted, so we run it only in a separate thread.
+		// since it sometimes refuses to shut down, we resort to the admittedly harsh
+		// means of killing the thread after a timeout.
+		PartitionInfoFetcher infoFetcher = new PartitionInfoFetcher(topic, config);
+		infoFetcher.start();
+		
+		KillerWatchDog watchDog = new KillerWatchDog(infoFetcher, 60000);
+		watchDog.start();
+		
+		final List<PartitionInfo> allPartitionsInTopic = infoFetcher.getPartitions();
+		
 		// brokers to fetch partitions from.
 		int fetchPartitionsCount = 0;
 		Map<Node, List<FetchPartition>> fetchBrokers = new HashMap<Node, List<FetchPartition>>();
-		for(PartitionInfo partitionInfo : allPartitionsInTopic) {
-			if(partitionInfo.leader() == null) {
-				throw new RuntimeException("Unable to consume partition "+partitionInfo.partition()+" from topic "+partitionInfo.topic()+" because it does not have a leader");
+		
+		for (PartitionInfo partitionInfo : allPartitionsInTopic) {
+			if (partitionInfo.leader() == null) {
+				throw new RuntimeException("Unable to consume partition " + partitionInfo.partition()
+						+ " from topic "+partitionInfo.topic()+" because it does not have a leader");
 			}
-			for(Map.Entry<TopicPartition, Long> partitionToRead: partitionsToRead.entrySet()) {
-				if(partitionToRead.getKey().partition() == partitionInfo.partition()) {
+			
+			for (Map.Entry<TopicPartition, Long> entry : partitionsToRead.entrySet()) {
+				final TopicPartition topicPartition = entry.getKey();
+				final long offset = entry.getValue();
+				
+				// check if that partition is for us
+				if (topicPartition.partition() == partitionInfo.partition()) {
 					List<FetchPartition> partitions = fetchBrokers.get(partitionInfo.leader());
-					if(partitions == null) {
+					if (partitions == null) {
 						partitions = new ArrayList<FetchPartition>();
+						fetchBrokers.put(partitionInfo.leader(), partitions);
 					}
-					FetchPartition fp = new FetchPartition();
-					fp.nextOffsetToRead = partitionToRead.getValue();
-					fp.partition = partitionToRead.getKey().partition();
-					partitions.add(fp);
+					
+					partitions.add(new FetchPartition(topicPartition.partition(), offset));
 					fetchPartitionsCount++;
-					fetchBrokers.put(partitionInfo.leader(), partitions);
+					
 				}
+				// else this partition is not for us
 			}
 		}
-		if(partitionsToRead.size() != fetchPartitionsCount) {
-			throw new RuntimeException(partitionsToRead.size() + " partitions to read, but got only "+fetchPartitionsCount+" partition infos with lead brokers.");
+		
+		if (partitionsToRead.size() != fetchPartitionsCount) {
+			throw new RuntimeException(partitionsToRead.size() + " partitions to read, but got only "
+					+ fetchPartitionsCount + " partition infos with lead brokers.");
 		}
-		// Create a queue for the threads to communicate
-		int queueSize = Integer.valueOf(config.getProperty(QUEUE_SIZE_KEY, DEFAULT_QUEUE_SIZE));
-		LinkedBlockingQueue<Tuple2<MessageAndOffset, Integer>> messageQueue = new LinkedBlockingQueue<Tuple2<MessageAndOffset, Integer>>(queueSize);
 
 		// create SimpleConsumers for each broker
-		List<SimpleConsumerThread> consumers = new ArrayList<SimpleConsumerThread>(fetchBrokers.size());
-		for(Map.Entry<Node, List<FetchPartition>> brokerInfo: fetchBrokers.entrySet()) {
-			SimpleConsumerThread thread = new SimpleConsumerThread(this.config, topic, brokerInfo.getKey(), brokerInfo.getValue(), messageQueue);
+		ArrayList<SimpleConsumerThread<?>> consumers = new ArrayList<>(fetchBrokers.size());
+		
+		for (Map.Entry<Node, List<FetchPartition>> brokerInfo : fetchBrokers.entrySet()) {
+			final Node broker = brokerInfo.getKey();
+			final List<FetchPartition> partitionsList = brokerInfo.getValue();
+			
+			FetchPartition[] partitions = partitionsList.toArray(new FetchPartition[partitionsList.size()]);
+
+			SimpleConsumerThread<T> thread = new SimpleConsumerThread<T>(this, config, topic,
+					broker, partitions, sourceContext, valueDeserializer, lastOffsets);
+
+			thread.setName(String.format("SimpleConsumer - %s - broker-%s (%s:%d)",
+					taskName, broker.idString(), broker.host(), broker.port()));
 			thread.setDaemon(true);
-			thread.setName("KafkaConsumer-SimpleConsumer-" + brokerInfo.getKey().idString());
-			thread.start();
 			consumers.add(thread);
-			LOG.info("Starting thread "+thread.getName()+" for fetching from broker "+brokerInfo.getKey().host());
 		}
-
-		// read from queue:
-		while(running) {
-			try {
-				Tuple2<MessageAndOffset, Integer> msg = messageQueue.take();
-				ByteBuffer payload = msg.f0.message().payload();
-				byte[] valueByte = new byte[payload.limit()];
-				payload.get(valueByte);
-				T value = valueDeserializer.deserialize(valueByte);
-				synchronized (sourceContext.getCheckpointLock()) {
-					lastOffsets[msg.f1] = msg.f0.offset();
-					sourceContext.collect(value);
+		
+		// last check whether we should abort.
+		if (!running) {
+			return;
+		}
+		
+		// start all consumer threads
+		for (SimpleConsumerThread<?> t : consumers) {
+			LOG.info("Starting thread {}", t.getName());
+			t.start();
+		}
+		
+		// wait until all consumer threads are done, or until we are aborted, or until
+		// an error occurred in one of the fetcher threads
+		try {
+			boolean someConsumersRunning = true;
+			while (running && error.get() == null && someConsumersRunning) {
+				try {
+					// wait for the consumer threads. if an error occurs, we are interrupted
+					for (SimpleConsumerThread<?> t : consumers) {
+						t.join();
+					}
+	
+					// safety net
+					someConsumersRunning = false;
+					for (SimpleConsumerThread<?> t : consumers) {
+						someConsumersRunning |= t.isAlive();
+					}
 				}
-			} catch (InterruptedException e) {
-				LOG.info("Queue consumption thread got interrupted. Stopping consumption and interrupting other threads");
-				running = false;
-				for(SimpleConsumerThread t: consumers) {
-					t.interrupt();
+				catch (InterruptedException e) {
+					// ignore. we should notice what happened in the next loop check
 				}
 			}
-
-			// see how the consumer threads are doing:
-			for(SimpleConsumerThread t: consumers) {
-				if(t.getError() != null) {
-					throw new RuntimeException("Consumer thread "+t.getName()+" had an exception", t.getError());
-				}
+			
+			// make sure any asynchronous error is noticed
+			Throwable error = this.error.get();
+			if (error != null) {
+				throw new Exception(error.getMessage(), error);
 			}
 		}
-
-		for(SimpleConsumerThread t: consumers) {
-			t.close();
+		finally {
+			// make sure that in any case (completion, abort, error), all spawned threads are stopped
+			for (SimpleConsumerThread<?> t : consumers) {
+				if (t.isAlive()) {
+					t.cancel();
+				}
+			}
 		}
-		sourceContext.close();
-	}
-
-	@Override
-	public void stop() {
-		running = false;
 	}
-
-	@Override
-	public void commit(Map<TopicPartition, Long> offsetsToCommit) {
-		throw new UnsupportedOperationException("This fetcher does not support committing offsets");
+	
+	/**
+	 * Reports an error from a fetch thread. This will cause the main thread to see this error,
+	 * abort, and cancel all other fetch threads.
+	 * 
+	 * @param error The error to report.
+	 */
+	void onErrorInFetchThread(Throwable error) {
+		if (this.error.compareAndSet(null, error)) {
+			// we are the first to report an error
+			if (mainThread != null) {
+				mainThread.interrupt();
+			}
+		}
 	}
 
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Representation of a partition to fetch.
 	 */
 	private static class FetchPartition {
-		/**
-		 * ID of the partition within the topic (0 indexed, as given by Kafka)
-		 */
-		public int partition;
-		/**
-		 * Offset pointing at the next element to read from that partition.
-		 */
-		public long nextOffsetToRead;
-
+		
+		/** ID of the partition within the topic (0 indexed, as given by Kafka) */
+		int partition;
+		
+		/** Offset pointing at the next element to read from that partition. */
+		long nextOffsetToRead;
+
+		FetchPartition(int partition, long nextOffsetToRead) {
+			this.partition = partition;
+			this.nextOffsetToRead = nextOffsetToRead;
+		}
+		
 		@Override
 		public String toString() {
-			return "FetchPartition{" +
-					"partition=" + partition +
-					", offset=" + nextOffsetToRead +
-					'}';
+			return "FetchPartition {partition=" + partition + ", offset=" + nextOffsetToRead + '}';
 		}
 	}
 
-	// --------------------------  Thread for a connection to a broker --------------------------
-
-	private static class SimpleConsumerThread extends Thread {
-
-		private final SimpleConsumer consumer;
-		private final List<FetchPartition> partitions;
-		private final LinkedBlockingQueue<Tuple2<MessageAndOffset, Integer>> messageQueue;
-		private final String clientId;
+	// ------------------------------------------------------------------------
+	//  Per broker fetcher
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Each broker needs its separate connection. This thread implements the connection to
+	 * one broker. The connection can fetch multiple partitions from the broker.
+	 * 
+	 * @param <T> The data type fetched.
+	 */
+	private static class SimpleConsumerThread<T> extends Thread {
+		
+		private final SourceFunction.SourceContext<T> sourceContext;
+		private final DeserializationSchema<T> valueDeserializer;
+		private final long[] offsetsState;
+		
+		private final FetchPartition[] partitions;
+		
+		private final Node broker;
 		private final String topic;
+		private final Properties config;
 
-		private final int fetchSize;
-		private final int maxWait;
-		private final int minBytes;
+		private final LegacyFetcher owner;
 
-		private boolean running = true;
-		private Throwable error = null;
+		private SimpleConsumer consumer;
+		
+		private volatile boolean running = true;
 
 
 		// exceptions are thrown locally
-		public SimpleConsumerThread(Properties config, String topic, Node leader, List<FetchPartition> partitions, LinkedBlockingQueue<Tuple2<MessageAndOffset, Integer>> messageQueue) {
-			Preconditions.checkNotNull(leader, "Leader can not be null");
-			Preconditions.checkNotNull(config, "The config properties can not be null");
-			// these are the actual configuration values of Kafka + their original default values.
-			int soTimeout = Integer.valueOf(config.getProperty("socket.timeout.ms", "30000"));
-			int bufferSize = Integer.valueOf(config.getProperty("socket.receive.buffer.bytes", "65536"));
-
-			this.fetchSize = Integer.valueOf(config.getProperty("fetch.message.max.bytes", "1048576"));
-			this.maxWait = Integer.valueOf(config.getProperty("fetch.wait.max.ms", "100"));
-			this.minBytes = Integer.valueOf(config.getProperty("fetch.min.bytes", "1"));
-
+		public SimpleConsumerThread(LegacyFetcher owner,
+									Properties config, String topic,
+									Node broker,
+									FetchPartition[] partitions,
+									SourceFunction.SourceContext<T> sourceContext,
+									DeserializationSchema<T> valueDeserializer,
+									long[] offsetsState) {
+			this.owner = owner;
+			this.config = config;
 			this.topic = topic;
+			this.broker = broker;
 			this.partitions = partitions;
-			this.messageQueue = messageQueue;
-			this.clientId = "flink-kafka-consumer-legacy-" + leader.idString();
-			// create consumer
-			consumer = new SimpleConsumer(leader.host(), leader.port(), bufferSize, soTimeout, clientId);
-
-			// list of partitions for which we need to get offsets (this is only effective if the offset is really not initialized
-			List<FetchPartition> getOffsetPartitions = new ArrayList<FetchPartition>();
-			for (FetchPartition fp : partitions) {
-				if (fp.nextOffsetToRead == FlinkKafkaConsumerBase.OFFSET_NOT_SET) {
-					// retrieve the offset from the consumer
-					getOffsetPartitions.add(fp);
-				}
-			}
-			if (getOffsetPartitions.size() > 0) {
-				long timeType = 0;
-				if (config.getProperty("auto.offset.reset", "latest").equals("latest")) {
-					timeType = OffsetRequest.LatestTime();
-				} else {
-					timeType = OffsetRequest.EarliestTime();
-				}
-				getLastOffset(consumer, topic, getOffsetPartitions, timeType);
-				LOG.info("No offsets found for topic " + topic + ", fetched the following start offsets {}", getOffsetPartitions);
-			}
+			this.sourceContext = checkNotNull(sourceContext);
+			this.valueDeserializer = checkNotNull(valueDeserializer);
+			this.offsetsState = checkNotNull(offsetsState);
 		}
 
 		@Override
 		public void run() {
 			try {
+				// set up the config values
+				final String clientId = "flink-kafka-consumer-legacy-" + broker.idString();
+
+				// these are the actual configuration values of Kafka + their original default values.
+				
+				final int soTimeout = Integer.valueOf(config.getProperty("socket.timeout.ms", "30000"));
+				final int bufferSize = Integer.valueOf(config.getProperty("socket.receive.buffer.bytes", "65536"));
+				final int fetchSize = Integer.valueOf(config.getProperty("fetch.message.max.bytes", "1048576"));
+				final int maxWait = Integer.valueOf(config.getProperty("fetch.wait.max.ms", "100"));
+				final int minBytes = Integer.valueOf(config.getProperty("fetch.min.bytes", "1"));
+				
+				// create the Kafka consumer that we actually use for fetching
+				consumer = new SimpleConsumer(broker.host(), broker.port(), bufferSize, soTimeout, clientId);
+
+				// make sure that all partitions have some offsets to start with
+				// those partitions that do not have an offset from a checkpoint need to get
+				// their start offset from ZooKeeper
+				
+				List<FetchPartition> partitionsToGetOffsetsFor = new ArrayList<FetchPartition>();
+
+				for (FetchPartition fp : partitions) {
+					if (fp.nextOffsetToRead == FlinkKafkaConsumer.OFFSET_NOT_SET) {
+						// retrieve the offset from the consumer
+						partitionsToGetOffsetsFor.add(fp);
+					}
+				}
+				if (partitionsToGetOffsetsFor.size() > 0) {
+					long timeType;
+					if (config.getProperty("auto.offset.reset", "latest").equals("latest")) {
+						timeType = OffsetRequest.LatestTime();
+					} else {
+						timeType = OffsetRequest.EarliestTime();
+					}
+					getLastOffset(consumer, topic, partitionsToGetOffsetsFor, timeType);
+					LOG.info("No prior offsets found for some partitions in topic {}. Fetched the following start offsets {}",
+							topic, partitionsToGetOffsetsFor);
+				}
+				
+				// Now, the actual work starts :-)
+				
 				while (running) {
 					FetchRequestBuilder frb = new FetchRequestBuilder();
-					frb.clientId(this.clientId);
+					frb.clientId(clientId);
 					frb.maxWait(maxWait);
 					frb.minBytes(minBytes);
+					
 					for (FetchPartition fp : partitions) {
-						frb.addFetch(topic, fp.partition, fp.nextOffsetToRead, this.fetchSize);
+						frb.addFetch(topic, fp.partition, fp.nextOffsetToRead, fetchSize);
 					}
 					kafka.api.FetchRequest fetchRequest = frb.build();
 					LOG.debug("Issuing fetch request {}", fetchRequest);
 
-					FetchResponse fetchResponse = null;
+					FetchResponse fetchResponse;
 					fetchResponse = consumer.fetch(fetchRequest);
 
-
 					if (fetchResponse.hasError()) {
 						String exception = "";
 						for (FetchPartition fp : partitions) {
 							short code;
 							if ((code = fetchResponse.errorCode(topic, fp.partition)) != ErrorMapping.NoError()) {
-								exception += "\nException for partition " + fp.partition + ": " + StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
+								exception += "\nException for partition " + fp.partition + ": " + 
+										StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
 							}
 						}
-						throw new RuntimeException("Error while fetching from broker: " + exception);
+						throw new IOException("Error while fetching from broker: " + exception);
 					}
 
 					int messagesInFetch = 0;
 					for (FetchPartition fp : partitions) {
-						ByteBufferMessageSet messageSet = fetchResponse.messageSet(topic, fp.partition);
+						final ByteBufferMessageSet messageSet = fetchResponse.messageSet(topic, fp.partition);
+						final int partition = fp.partition;
+						
 						for (MessageAndOffset msg : messageSet) {
-							messagesInFetch++;
-							try {
+							if (running) {
+								messagesInFetch++;
 								if (msg.offset() < fp.nextOffsetToRead) {
-									LOG.info("Skipping message with offset " + msg.offset() + " because we have seen messages until " + fp.nextOffsetToRead + " from partition " + fp.partition + " already");
 									// we have seen this message already
+									LOG.info("Skipping message with offset " + msg.offset()
+											+ " because we have seen messages until " + fp.nextOffsetToRead
+											+ " from partition " + fp.partition + " already");
 									continue;
 								}
-								messageQueue.put(new Tuple2<MessageAndOffset, Integer>(msg, fp.partition));
-								fp.nextOffsetToRead = msg.offset() + 1; // advance offset for the next request
-							} catch (InterruptedException e) {
-								LOG.debug("Consumer thread got interrupted. Stopping consumption");
-								running = false;
+								
+								ByteBuffer payload = msg.message().payload();
+								byte[] valueByte = new byte[payload.remaining()];
+								payload.get(valueByte);
+								
+								final T value = valueDeserializer.deserialize(valueByte);
+								final long offset = msg.offset();
+										
+								synchronized (sourceContext.getCheckpointLock()) {
+									offsetsState[partition] = offset;
+									sourceContext.collect(value);
+								}
+								
+								// advance offset for the next request
+								fp.nextOffsetToRead = offset + 1;
+							}
+							else {
+								// no longer running
+								return;
 							}
 						}
 					}
 					LOG.debug("This fetch contained {} messages", messagesInFetch);
 				}
-			} catch(Throwable cause) {
-				this.error = new RuntimeException("Error while reading data in thread "+this.getName(), cause);
-			} finally {
+			}
+			catch (Throwable t) {
+				// report to the main thread
+				owner.onErrorInFetchThread(t);
+			}
+			finally {
 				// end of run loop. close connection to consumer
-				consumer.close();
+				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);
+					}
+				}
 			}
-
 		}
 
-		public void close() {
-			running = false;
-			consumer.close();
+		/**
+		 * Cancels this fetch thread. The thread will release all resources and terminate.
+		 */
+		public void cancel() {
+			this.running = false;
+			
+			// interrupt whatever the consumer is doing
+			if (consumer != null) {
+				consumer.close();
+			}
+			
+			this.interrupt();
 		}
 
-		public Throwable getError() {
-			return error;
+		/**
+		 * Request latest offsets for a set of partitions, via a Kafka consumer.
+		 *
+		 * @param consumer The consumer connected to lead broker
+		 * @param topic The topic name
+		 * @param partitions The list of partitions we need offsets for
+		 * @param whichTime The type of time we are requesting. -1 and -2 are special constants (See OffsetRequest)
+		 */
+		private static void getLastOffset(SimpleConsumer consumer, String topic, List<FetchPartition> partitions, long whichTime) {
+
+			Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo = new HashMap<TopicAndPartition, PartitionOffsetRequestInfo>();
+			for (FetchPartition fp: partitions) {
+				TopicAndPartition topicAndPartition = new TopicAndPartition(topic, fp.partition);
+				requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo(whichTime, 1));
+			}
+
+			kafka.javaapi.OffsetRequest request = new kafka.javaapi.OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId());
+			OffsetResponse response = consumer.getOffsetsBefore(request);
+
+			if (response.hasError()) {
+				String exception = "";
+				for (FetchPartition fp: partitions) {
+					short code;
+					if ( (code=response.errorCode(topic, fp.partition)) != ErrorMapping.NoError()) {
+						exception += "\nException for partition "+fp.partition+": "+ StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
+					}
+				}
+				throw new RuntimeException("Unable to get last offset for topic " + topic + " and partitions " + partitions
+						+ ". " + exception);
+			}
+
+			for (FetchPartition fp: partitions) {
+				// the resulting offset is the next offset we are going to read
+				// for not-yet-consumed partitions, it is 0.
+				fp.nextOffsetToRead = response.offsets(topic, fp.partition)[0];
+			}
 		}
 	}
+	
+	private static class PartitionInfoFetcher extends Thread {
 
-	/**
-	 * Request latest offsets from Kafka.
-	 *
-	 * @param consumer consumer connected to lead broker
-	 * @param topic topic name
-	 * @param partitions list of partitions we need offsets for
-	 * @param whichTime type of time we are requesting. -1 and -2 are special constants (See OffsetRequest)
-	 */
-	private static void getLastOffset(SimpleConsumer consumer, String topic, List<FetchPartition> partitions, long whichTime) {
+		private final String topic;
+		private final Properties properties;
+		
+		private volatile List<PartitionInfo> result;
+		private volatile Throwable error;
 
-		Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo = new HashMap<TopicAndPartition, PartitionOffsetRequestInfo>();
-		for(FetchPartition fp: partitions) {
-			TopicAndPartition topicAndPartition = new TopicAndPartition(topic, fp.partition);
-			requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo(whichTime, 1));
+		
+		PartitionInfoFetcher(String topic, Properties properties) {
+			this.topic = topic;
+			this.properties = properties;
 		}
 
-		kafka.javaapi.OffsetRequest request = new kafka.javaapi.OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId());
-		OffsetResponse response = consumer.getOffsetsBefore(request);
-
-		if (response.hasError()) {
-			String exception = "";
-			for(FetchPartition fp: partitions) {
-				short code;
-				if( (code=response.errorCode(topic, fp.partition)) != ErrorMapping.NoError()) {
-					exception += "\nException for partition "+fp.partition+": "+ StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
-				}
+		@Override
+		public void run() {
+			try {
+				result = FlinkKafkaConsumer.getPartitionsForTopic(topic, properties);
+			}
+			catch (Throwable t) {
+				this.error = t;
+			}
+		}
+		
+		public List<PartitionInfo> getPartitions() throws Exception {
+			try {
+				this.join();
 			}
-			throw new RuntimeException("Unable to get last offset for topic "+topic+" and partitions "+partitions +". "+exception);
+			catch (InterruptedException e) {
+				throw new Exception("Partition fetching was cancelled before completion");
+			}
+			
+			if (error != null) {
+				throw new Exception("Failed to fetch partitions for topic " + topic, error);
+			}
+			if (result != null) {
+				return result;
+			}
+			throw new Exception("Partition fetching failed");
 		}
+	}
 
-		for(FetchPartition fp: partitions) {
-			// the resulting offset is the next offset we are going to read
-			// for not-yet-consumed partitions, it is 0.
-			fp.nextOffsetToRead = response.offsets(topic, fp.partition)[0];
+	private static class KillerWatchDog extends Thread {
+		
+		private final Thread toKill;
+		private final long timeout;
+
+		private KillerWatchDog(Thread toKill, long timeout) {
+			super("KillerWatchDog");
+			setDaemon(true);
+			
+			this.toKill = toKill;
+			this.timeout = timeout;
 		}
 
+		@SuppressWarnings("deprecation")
+		@Override
+		public void run() {
+			final long deadline = System.currentTimeMillis() + timeout;
+			long now;
+			
+			while (toKill.isAlive() && (now = System.currentTimeMillis()) < deadline) {
+				try {
+					toKill.join(deadline - now);
+				}
+				catch (InterruptedException e) {
+					// ignore here, out 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/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/NewConsumerApiFetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/NewConsumerApiFetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/NewConsumerApiFetcher.java
new file mode 100644
index 0000000..db9424e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/NewConsumerApiFetcher.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.internals;
+
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.kafka_backport.clients.consumer.CommitType;
+import org.apache.flink.kafka_backport.clients.consumer.ConsumerRecord;
+import org.apache.flink.kafka_backport.clients.consumer.ConsumerRecords;
+import org.apache.flink.kafka_backport.clients.consumer.KafkaConsumer;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.serialization.ByteArrayDeserializer;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * A fetcher that uses the new Kafka consumer API to fetch data for a specifies set of partitions.
+ */
+public class NewConsumerApiFetcher implements Fetcher, OffsetHandler {
+
+	private static final String POLL_TIMEOUT_PROPERTY = "flink.kafka.consumer.poll.timeout";
+	private static final long DEFAULT_POLL_TIMEOUT = 50;
+	
+	private static final ByteArrayDeserializer NO_OP_SERIALIZER = new ByteArrayDeserializer();
+
+	
+	private final KafkaConsumer<byte[], byte[]> fetcher;
+	
+	private final long pollTimeout;
+	
+	private volatile boolean running = true;
+
+	
+	public NewConsumerApiFetcher(Properties props) {
+		this.pollTimeout = props.contains(POLL_TIMEOUT_PROPERTY) ?
+				Long.valueOf(props.getProperty(POLL_TIMEOUT_PROPERTY)) :
+				DEFAULT_POLL_TIMEOUT;
+		
+		this.fetcher = new KafkaConsumer<byte[], byte[]>(props, null, NO_OP_SERIALIZER, NO_OP_SERIALIZER);
+	}
+
+	@Override
+	public void setPartitionsToRead(List<TopicPartition> partitions) {
+		synchronized (fetcher) {
+			if (fetcher.subscriptions().isEmpty()) {
+				fetcher.subscribe(partitions.toArray(new TopicPartition[partitions.size()]));
+			}
+			else {
+				throw new IllegalStateException("Fetcher has already subscribed to its set of partitions");
+			}
+		}
+	}
+
+	@Override
+	public void seek(TopicPartition topicPartition, long offsetToRead) {
+		synchronized (fetcher) {
+			fetcher.seek(topicPartition, offsetToRead);
+		}
+	}
+
+	@Override
+	public void close() {
+		running = false;
+		synchronized (fetcher) {
+			fetcher.close();
+		}
+	}
+
+	@Override
+	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
+						DeserializationSchema<T> valueDeserializer, long[] lastOffsets) {
+		while (running) {
+			// poll is always returning a new object.
+			ConsumerRecords<byte[], byte[]> consumed;
+			synchronized (fetcher) {
+				consumed = fetcher.poll(pollTimeout);
+			}
+
+			final Iterator<ConsumerRecord<byte[], byte[]>> records = consumed.iterator();
+			while (running && records.hasNext()) {
+				ConsumerRecord<byte[], byte[]> record = records.next();
+				T value = valueDeserializer.deserialize(record.value());
+				
+				// synchronize inside the loop to allow checkpoints in between batches
+				synchronized (sourceContext.getCheckpointLock()) {
+					sourceContext.collect(value);
+					lastOffsets[record.partition()] = record.offset();
+				}
+			}
+		}
+	}
+
+	@Override
+	public void commit(Map<TopicPartition, Long> offsetsToCommit) {
+		synchronized (fetcher) {
+			fetcher.commit(offsetsToCommit, CommitType.SYNC);
+		}
+	}
+
+	@Override
+	public void seekFetcherToInitialOffsets(List<TopicPartition> partitions, Fetcher fetcher) {
+		// no need to do anything here.
+		// if Kafka manages the offsets, it has them automatically
+	}
+}

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka081ITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka081ITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka081ITCase.java
index 9a20186..218315f 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka081ITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka081ITCase.java
@@ -14,56 +14,87 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.streaming.connectors;
 
-import org.apache.flink.streaming.connectors.internals.FlinkKafkaConsumerBase;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
-import java.util.Arrays;
+import org.junit.Test;
+
 import java.util.Properties;
 
 
-public class Kafka081ITCase extends KafkaTestBase {
+public class Kafka081ITCase extends KafkaConsumerTestBase {
+	
 	@Override
-	<T> FlinkKafkaConsumerBase<T> getConsumer(String topic, DeserializationSchema deserializationSchema, Properties props) {
-		return new TestFlinkKafkaConsumer081<T>(topic, deserializationSchema, props);
+	protected <T> FlinkKafkaConsumer<T> getConsumer(String topic, DeserializationSchema<T> deserializationSchema, Properties props) {
+		return new FlinkKafkaConsumer081<T>(topic, deserializationSchema, props);
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Suite of Tests
+	// ------------------------------------------------------------------------
+	
+	@Test
+	public void testCheckpointing() {
+		runCheckpointingTest();
 	}
 
-	@Override
-	long[] getFinalOffsets() {
-		return TestFlinkKafkaConsumer081.finalOffset;
+	@Test
+	public void testOffsetInZookeeper() {
+		runOffsetInZookeeperValidationTest();
+	}
+	
+	@Test
+	public void testConcurrentProducerConsumerTopology() {
+		runSimpleConcurrentProducerConsumerTopology();
 	}
 
-	@Override
-	void resetOffsets() {
-		TestFlinkKafkaConsumer081.finalOffset = null;
+	// --- canceling / failures ---
+	
+	@Test
+	public void testCancelingEmptyTopic() {
+		runCancelingOnEmptyInputTest();
+	}
+
+	@Test
+	public void testCancelingFullTopic() {
+		runCancelingOnFullInputTest();
 	}
 
+	@Test
+	public void testFailOnDeploy() {
+		runFailOnDeployTest();
+	}
 
-	public static class TestFlinkKafkaConsumer081<OUT> extends FlinkKafkaConsumer081<OUT> {
-		public static long[] finalOffset;
-		public TestFlinkKafkaConsumer081(String topicName, DeserializationSchema<OUT> deserializationSchema, Properties consumerConfig) {
-			super(topicName, deserializationSchema, consumerConfig);
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			synchronized (commitedOffsets) {
-				LOG.info("Setting final offset from "+ Arrays.toString(commitedOffsets));
-				if (finalOffset == null) {
-					finalOffset = new long[commitedOffsets.length];
-				}
-				for(int i = 0; i < commitedOffsets.length; i++) {
-					if(commitedOffsets[i] > 0) {
-						if(finalOffset[i] > 0) {
-							throw new RuntimeException("This is unexpected on i = "+i);
-						}
-						finalOffset[i] = commitedOffsets[i];
-					}
-				}
-			}
-		}
+	// --- source to partition mappings and exactly once ---
+	
+	@Test
+	public void testOneToOneSources() {
+		runOneToOneExactlyOnceTest();
 	}
 
+	@Test
+	public void testOneSourceMultiplePartitions() {
+		runOneSourceMultiplePartitionsExactlyOnceTest();
+	}
+
+	@Test
+	public void testMultipleSourcesOnePartition() {
+		runMultipleSourcesOnePartitionExactlyOnceTest();
+	}
+
+	// --- broker failure ---
+
+	@Test
+	public void testBrokerFailure() {
+		runBrokerFailureTest();
+	}
+
+	// --- special executions ---
+	
+	@Test
+	public void testBigRecordJob() {
+		runBigRecordTestTopology();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka082ITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka082ITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka082ITCase.java
index 43cd0f9..2f80fcb 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka082ITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka082ITCase.java
@@ -14,57 +14,89 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.streaming.connectors;
 
-import org.apache.flink.streaming.connectors.internals.FlinkKafkaConsumerBase;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
-import java.util.Arrays;
+import org.junit.Ignore;
+import org.junit.Test;
+
 import java.util.Properties;
 
 
-public class Kafka082ITCase extends KafkaTestBase {
+public class Kafka082ITCase extends KafkaConsumerTestBase {
+	
 	@Override
-	<T> FlinkKafkaConsumerBase<T> getConsumer(String topic, DeserializationSchema deserializationSchema, Properties props) {
-		return new TestFlinkKafkaConsumer082<T>(topic, deserializationSchema, props);
+	protected <T> FlinkKafkaConsumer<T> getConsumer(String topic, DeserializationSchema<T> deserializationSchema, Properties props) {
+		return new FlinkKafkaConsumer082<T>(topic, deserializationSchema, props);
 	}
 
-	@Override
-	long[] getFinalOffsets() {
-		return TestFlinkKafkaConsumer082.finalOffset;
+	// ------------------------------------------------------------------------
+	//  Suite of Tests
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testCheckpointing() {
+		runCheckpointingTest();
 	}
 
-	@Override
-	void resetOffsets() {
-		TestFlinkKafkaConsumer082.finalOffset = null;
+	@Test
+	public void testOffsetInZookeeper() {
+		runOffsetInZookeeperValidationTest();
+	}
+
+	@Test
+	public void testConcurrentProducerConsumerTopology() {
+		runSimpleConcurrentProducerConsumerTopology();
+	}
+
+	// --- canceling / failures ---
+
+	@Test
+	public void testCancelingEmptyTopic() {
+		runCancelingOnEmptyInputTest();
 	}
 
+	@Test
+	public void testCancelingFullTopic() {
+		runCancelingOnFullInputTest();
+	}
 
-	public static class TestFlinkKafkaConsumer082<OUT> extends FlinkKafkaConsumer082<OUT> {
-		private final static Object sync = new Object();
-		public static long[] finalOffset;
-		public TestFlinkKafkaConsumer082(String topicName, DeserializationSchema<OUT> deserializationSchema, Properties consumerConfig) {
-			super(topicName, deserializationSchema, consumerConfig);
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			synchronized (commitedOffsets) {
-				LOG.info("Setting final offset from "+ Arrays.toString(commitedOffsets));
-				if (finalOffset == null) {
-					finalOffset = new long[commitedOffsets.length];
-				}
-				for(int i = 0; i < commitedOffsets.length; i++) {
-					if(commitedOffsets[i] > 0) {
-						if(finalOffset[i] > 0) {
-							throw new RuntimeException("This is unexpected on i = "+i);
-						}
-						finalOffset[i] = commitedOffsets[i];
-					}
-				}
-			}
-		}
+	@Test
+	public void testFailOnDeploy() {
+		runFailOnDeployTest();
 	}
 
+	// --- source to partition mappings and exactly once ---
+
+	@Test
+	public void testOneToOneSources() {
+		runOneToOneExactlyOnceTest();
+	}
+
+	@Test
+	public void testOneSourceMultiplePartitions() {
+		runOneSourceMultiplePartitionsExactlyOnceTest();
+	}
+
+	@Test
+	public void testMultipleSourcesOnePartition() {
+		runMultipleSourcesOnePartitionExactlyOnceTest();
+	}
+
+	// --- broker failure ---
+
+	@Test
+	public void testBrokerFailure() {
+		runBrokerFailureTest();
+	}
+
+	// --- special executions ---
+
+	@Test
+	@Ignore("this does not work with the new consumer")
+	public void testBigRecordJob() {
+		runBigRecordTestTopology();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka083ITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka083ITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka083ITCase.java
deleted file mode 100644
index f7933f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka083ITCase.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;
-
-import org.apache.flink.streaming.connectors.internals.FlinkKafkaConsumerBase;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.junit.Ignore;
-
-import java.util.Arrays;
-import java.util.Properties;
-
-
-public class Kafka083ITCase extends KafkaTestBase {
-	@Override
-	<T> FlinkKafkaConsumerBase<T> getConsumer(String topic, DeserializationSchema deserializationSchema, Properties props) {
-		return new TestFlinkKafkaConsumer083<T>(topic, deserializationSchema, props);
-	}
-
-	@Override
-	long[] getFinalOffsets() {
-		return TestFlinkKafkaConsumer083.finalOffset;
-	}
-
-	@Override
-	void resetOffsets() {
-		TestFlinkKafkaConsumer083.finalOffset = null;
-	}
-
-
-	public static class TestFlinkKafkaConsumer083<OUT> extends FlinkKafkaConsumer083<OUT> {
-		public static long[] finalOffset;
-		public TestFlinkKafkaConsumer083(String topicName, DeserializationSchema<OUT> deserializationSchema, Properties consumerConfig) {
-			super(topicName, deserializationSchema, consumerConfig);
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			synchronized (commitedOffsets) {
-				LOG.info("Setting final offset from "+ Arrays.toString(commitedOffsets));
-				if (finalOffset == null) {
-					finalOffset = new long[commitedOffsets.length];
-				}
-				for(int i = 0; i < commitedOffsets.length; i++) {
-					if(commitedOffsets[i] > 0) {
-						if(finalOffset[i] > 0) {
-							throw new RuntimeException("This is unexpected on i = "+i);
-						}
-						finalOffset[i] = commitedOffsets[i];
-					}
-				}
-			}
-		}
-	}
-
-	@Ignore
-	@Override
-	public void brokerFailureTest() throws Exception {
-		// Skipping test: The test is committing the offsets to the Kafka Broker.
-		// only 0.8.3 brokers support that.
-		return;
-	}
-
-	@Ignore
-	@Override
-	public void testFlinkKafkaConsumerWithOffsetUpdates() throws Exception {
-		// Skipping test (see above)
-		return;
-	}
-}

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

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


[16/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Protocol.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Protocol.java
new file mode 100644
index 0000000..f7c8981
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Protocol.java
@@ -0,0 +1,474 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.protocol;
+
+import org.apache.flink.kafka_backport.common.protocol.types.ArrayOf;
+import org.apache.flink.kafka_backport.common.protocol.types.Field;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Type;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class Protocol {
+
+    public static final Schema REQUEST_HEADER = new Schema(new Field("api_key", Type.INT16, "The id of the request type."),
+                                                           new Field("api_version", Type.INT16, "The version of the API."),
+                                                           new Field("correlation_id",
+                                                                     Type.INT32,
+                                                                     "A user-supplied integer value that will be passed back with the response"),
+                                                           new Field("client_id",
+                                                                     Type.STRING,
+                                                                     "A user specified identifier for the client making the request."));
+
+    public static final Schema RESPONSE_HEADER = new Schema(new Field("correlation_id",
+                                                                      Type.INT32,
+                                                                      "The user-supplied value passed in with the request"));
+
+    /* Metadata api */
+
+    public static final Schema METADATA_REQUEST_V0 = new Schema(new Field("topics",
+                                                                          new ArrayOf(Type.STRING),
+                                                                          "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics."));
+
+    public static final Schema BROKER = new Schema(new Field("node_id", Type.INT32, "The broker id."),
+                                                   new Field("host", Type.STRING, "The hostname of the broker."),
+                                                   new Field("port",
+                                                             Type.INT32,
+                                                             "The port on which the broker accepts requests."));
+
+    public static final Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code",
+                                                                            Type.INT16,
+                                                                            "The error code for the partition, if any."),
+                                                                  new Field("partition_id",
+                                                                            Type.INT32,
+                                                                            "The id of the partition."),
+                                                                  new Field("leader",
+                                                                            Type.INT32,
+                                                                            "The id of the broker acting as leader for this partition."),
+                                                                  new Field("replicas",
+                                                                            new ArrayOf(Type.INT32),
+                                                                            "The set of all nodes that host this partition."),
+                                                                  new Field("isr",
+                                                                            new ArrayOf(Type.INT32),
+                                                                            "The set of nodes that are in sync with the leader for this partition."));
+
+    public static final Schema TOPIC_METADATA_V0 = new Schema(new Field("topic_error_code",
+                                                                        Type.INT16,
+                                                                        "The error code for the given topic."),
+                                                              new Field("topic", Type.STRING, "The name of the topic"),
+                                                              new Field("partition_metadata",
+                                                                        new ArrayOf(PARTITION_METADATA_V0),
+                                                                        "Metadata for each partition of the topic."));
+
+    public static final Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers",
+                                                                           new ArrayOf(BROKER),
+                                                                           "Host and port information for all brokers."),
+                                                                 new Field("topic_metadata",
+                                                                           new ArrayOf(TOPIC_METADATA_V0)));
+
+    public static final Schema[] METADATA_REQUEST = new Schema[] {METADATA_REQUEST_V0};
+    public static final Schema[] METADATA_RESPONSE = new Schema[] {METADATA_RESPONSE_V0};
+
+    /* Produce api */
+
+    public static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", Type.STRING),
+                                                                  new Field("data", new ArrayOf(new Schema(new Field("partition", Type.INT32),
+                                                                                                     new Field("record_set", Type.BYTES)))));
+
+    public static final Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks",
+                                                                   Type.INT16,
+                                                                   "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."),
+                                                               new Field("timeout", Type.INT32, "The time to await a response in ms."),
+                                                               new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0)));
+
+    public static final Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                    new ArrayOf(new Schema(new Field("topic", Type.STRING),
+                                                                                           new Field("partition_responses",
+                                                                                                     new ArrayOf(new Schema(new Field("partition",
+                                                                                                                                      Type.INT32),
+                                                                                                                            new Field("error_code",
+                                                                                                                                      Type.INT16),
+                                                                                                                            new Field("base_offset",
+                                                                                                                                      Type.INT64))))))));
+
+    public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0};
+    public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0};
+
+    /* Offset commit api */
+    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                         Type.INT32,
+                                                                                         "Topic partition id."),
+                                                                               new Field("offset",
+                                                                                         Type.INT64,
+                                                                                         "Message offset to be committed."),
+                                                                               new Field("metadata",
+                                                                                         Type.STRING,
+                                                                                         "Any associated metadata the client wants to keep."));
+
+    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition",
+                                                                                         Type.INT32,
+                                                                                         "Topic partition id."),
+                                                                               new Field("offset",
+                                                                                         Type.INT64,
+                                                                                         "Message offset to be committed."),
+                                                                               new Field("timestamp",
+                                                                                         Type.INT64,
+                                                                                         "Timestamp of the commit"),
+                                                                               new Field("metadata",
+                                                                                         Type.STRING,
+                                                                                         "Any associated metadata the client wants to keep."));
+
+    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V2 = new Schema(new Field("partition",
+                                                                                         Type.INT32,
+                                                                                         "Topic partition id."),
+                                                                               new Field("offset",
+                                                                                         Type.INT64,
+                                                                                         "Message offset to be committed."),
+                                                                               new Field("metadata",
+                                                                                         Type.STRING,
+                                                                                         "Any associated metadata the client wants to keep."));
+
+    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
+                                                                                     Type.STRING,
+                                                                                     "Topic to commit."),
+                                                                           new Field("partitions",
+                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0),
+                                                                                     "Partitions to commit offsets."));
+
+    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(new Field("topic",
+                                                                                     Type.STRING,
+                                                                                     "Topic to commit."),
+                                                                           new Field("partitions",
+                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1),
+                                                                                     "Partitions to commit offsets."));
+
+    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V2 = new Schema(new Field("topic",
+                                                                                     Type.STRING,
+                                                                                     "Topic to commit."),
+                                                                           new Field("partitions",
+                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V2),
+                                                                                     "Partitions to commit offsets."));
+
+    public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                               Type.STRING,
+                                                                               "The consumer group id."),
+                                                                     new Field("topics",
+                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
+                                                                               "Topics to commit offsets."));
+
+    public static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id",
+                                                                               Type.STRING,
+                                                                               "The consumer group id."),
+                                                                     new Field("group_generation_id",
+                                                                               Type.INT32,
+                                                                               "The generation of the consumer group."),
+                                                                     new Field("consumer_id",
+                                                                               Type.STRING,
+                                                                               "The consumer id assigned by the group coordinator."),
+                                                                     new Field("topics",
+                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1),
+                                                                               "Topics to commit offsets."));
+
+    public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id",
+                                                                               Type.STRING,
+                                                                               "The consumer group id."),
+                                                                     new Field("group_generation_id",
+                                                                               Type.INT32,
+                                                                               "The generation of the consumer group."),
+                                                                     new Field("consumer_id",
+                                                                               Type.STRING,
+                                                                               "The consumer id assigned by the group coordinator."),
+                                                                     new Field("retention_time",
+                                                                               Type.INT64,
+                                                                               "Time period in ms to retain the offset."),
+                                                                     new Field("topics",
+                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2),
+                                                                               "Topics to commit offsets."));
+
+    public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                          Type.INT32,
+                                                                                          "Topic partition id."),
+                                                                                new Field("error_code",
+                                                                                          Type.INT16));
+
+    public static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
+                                                                            new Field("partition_responses",
+                                                                                      new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0)));
+
+    public static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                                new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
+
+    public static final Schema[] OFFSET_COMMIT_REQUEST = new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2};
+
+    /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */
+    public static final Schema OFFSET_COMMIT_RESPONSE_V1 = OFFSET_COMMIT_RESPONSE_V0;
+    public static final Schema OFFSET_COMMIT_RESPONSE_V2 = OFFSET_COMMIT_RESPONSE_V0;
+
+    public static final Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V1, OFFSET_COMMIT_RESPONSE_V2};
+
+    /* Offset fetch api */
+
+    /*
+     * Wire formats of version 0 and 1 are the same, but with different functionality.
+     * Version 0 will read the offsets from ZK;
+     * Version 1 will read the offsets from Kafka.
+     */
+    public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                        Type.INT32,
+                                                                                        "Topic partition id."));
+
+    public static final Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
+                                                                                    Type.STRING,
+                                                                                    "Topic to fetch offset."),
+                                                                          new Field("partitions",
+                                                                                    new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0),
+                                                                                    "Partitions to fetch offsets."));
+
+    public static final Schema OFFSET_FETCH_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                              Type.STRING,
+                                                                              "The consumer group id."),
+                                                                    new Field("topics",
+                                                                              new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0),
+                                                                              "Topics to fetch offsets."));
+
+    public static final Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                         Type.INT32,
+                                                                                         "Topic partition id."),
+                                                                               new Field("offset",
+                                                                                         Type.INT64,
+                                                                                         "Last committed message offset."),
+                                                                               new Field("metadata",
+                                                                                         Type.STRING,
+                                                                                         "Any associated metadata the client wants to keep."),
+                                                                               new Field("error_code", Type.INT16));
+
+    public static final Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
+                                                                           new Field("partition_responses",
+                                                                                     new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0)));
+
+    public static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                               new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)));
+
+    public static final Schema OFFSET_FETCH_REQUEST_V1 = OFFSET_FETCH_REQUEST_V0;
+    public static final Schema OFFSET_FETCH_RESPONSE_V1 = OFFSET_FETCH_RESPONSE_V0;
+
+    public static final Schema[] OFFSET_FETCH_REQUEST = new Schema[] {OFFSET_FETCH_REQUEST_V0, OFFSET_FETCH_REQUEST_V1};
+    public static final Schema[] OFFSET_FETCH_RESPONSE = new Schema[] {OFFSET_FETCH_RESPONSE_V0, OFFSET_FETCH_RESPONSE_V1};
+
+    /* List offset api */
+    public static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                       Type.INT32,
+                                                                                       "Topic partition id."),
+                                                                             new Field("timestamp", Type.INT64, "Timestamp."),
+                                                                             new Field("max_num_offsets",
+                                                                                       Type.INT32,
+                                                                                       "Maximum offsets to return."));
+
+    public static final Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
+                                                                                   Type.STRING,
+                                                                                   "Topic to list offset."),
+                                                                         new Field("partitions",
+                                                                                   new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0),
+                                                                                   "Partitions to list offset."));
+
+    public static final Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id",
+                                                                             Type.INT32,
+                                                                             "Broker id of the follower. For normal consumers, use -1."),
+                                                                   new Field("topics",
+                                                                             new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0),
+                                                                             "Topics to list offsets."));
+
+    public static final Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                        Type.INT32,
+                                                                                        "Topic partition id."),
+                                                                              new Field("error_code", Type.INT16),
+                                                                              new Field("offsets",
+                                                                                        new ArrayOf(Type.INT64),
+                                                                                        "A list of offsets."));
+
+    public static final Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
+                                                                          new Field("partition_responses",
+                                                                                    new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0)));
+
+    public static final Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                              new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0)));
+
+    public static final Schema[] LIST_OFFSET_REQUEST = new Schema[] {LIST_OFFSET_REQUEST_V0};
+    public static final Schema[] LIST_OFFSET_RESPONSE = new Schema[] {LIST_OFFSET_RESPONSE_V0};
+
+    /* Fetch api */
+    public static final Schema FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                 Type.INT32,
+                                                                                 "Topic partition id."),
+                                                                       new Field("fetch_offset",
+                                                                                 Type.INT64,
+                                                                                 "Message offset."),
+                                                                       new Field("max_bytes",
+                                                                                 Type.INT32,
+                                                                                 "Maximum bytes to fetch."));
+
+    public static final Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", Type.STRING, "Topic to fetch."),
+                                                                   new Field("partitions",
+                                                                             new ArrayOf(FETCH_REQUEST_PARTITION_V0),
+                                                                             "Partitions to fetch."));
+
+    public static final Schema FETCH_REQUEST_V0 = new Schema(new Field("replica_id",
+                                                                       Type.INT32,
+                                                                       "Broker id of the follower. For normal consumers, use -1."),
+                                                             new Field("max_wait_time",
+                                                                       Type.INT32,
+                                                                       "Maximum time in ms to wait for the response."),
+                                                             new Field("min_bytes",
+                                                                       Type.INT32,
+                                                                       "Minimum bytes to accumulate in the response."),
+                                                             new Field("topics",
+                                                                       new ArrayOf(FETCH_REQUEST_TOPIC_V0),
+                                                                       "Topics to fetch."));
+
+    public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                  Type.INT32,
+                                                                                  "Topic partition id."),
+                                                                        new Field("error_code", Type.INT16),
+                                                                        new Field("high_watermark",
+                                                                                  Type.INT64,
+                                                                                  "Last committed offset."),
+                                                                        new Field("record_set", Type.BYTES));
+
+    public static final Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
+                                                                    new Field("partition_responses",
+                                                                              new ArrayOf(FETCH_RESPONSE_PARTITION_V0)));
+
+    public static final Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                        new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
+
+    public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0};
+    public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0};
+
+    /* Consumer metadata api */
+    public static final Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                                   Type.STRING,
+                                                                                   "The consumer group id."));
+
+    public static final Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", Type.INT16),
+                                                                          new Field("coordinator",
+                                                                                    BROKER,
+                                                                                    "Host and port information for the coordinator for a consumer group."));
+
+    public static final Schema[] CONSUMER_METADATA_REQUEST = new Schema[] {CONSUMER_METADATA_REQUEST_V0};
+    public static final Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] {CONSUMER_METADATA_RESPONSE_V0};
+
+    /* Join group api */
+    public static final Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                            Type.STRING,
+                                                                            "The consumer group id."),
+                                                                  new Field("session_timeout",
+                                                                            Type.INT32,
+                                                                            "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."),
+                                                                  new Field("topics",
+                                                                            new ArrayOf(Type.STRING),
+                                                                            "An array of topics to subscribe to."),
+                                                                  new Field("consumer_id",
+                                                                            Type.STRING,
+                                                                            "The assigned consumer id or an empty string for a new consumer."),
+                                                                  new Field("partition_assignment_strategy",
+                                                                            Type.STRING,
+                                                                            "The strategy for the coordinator to assign partitions."));
+
+    public static final Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
+                                                                         new Field("partitions", new ArrayOf(Type.INT32)));
+    public static final Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", Type.INT16),
+                                                                   new Field("group_generation_id",
+                                                                             Type.INT32,
+                                                                             "The generation of the consumer group."),
+                                                                   new Field("consumer_id",
+                                                                             Type.STRING,
+                                                                             "The consumer id assigned by the group coordinator."),
+                                                                   new Field("assigned_partitions",
+                                                                             new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0)));
+
+    public static final Schema[] JOIN_GROUP_REQUEST = new Schema[] {JOIN_GROUP_REQUEST_V0};
+    public static final Schema[] JOIN_GROUP_RESPONSE = new Schema[] {JOIN_GROUP_RESPONSE_V0};
+
+    /* Heartbeat api */
+    public static final Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id", Type.STRING, "The consumer group id."),
+                                                                 new Field("group_generation_id",
+                                                                           Type.INT32,
+                                                                           "The generation of the consumer group."),
+                                                                 new Field("consumer_id",
+                                                                           Type.STRING,
+                                                                           "The consumer id assigned by the group coordinator."));
+
+    public static final Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", Type.INT16));
+
+    public static final Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0};
+    public static final Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0};
+
+    /* an array of all requests and responses with all schema versions */
+    public static final Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][];
+    public static final Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][];
+
+    /* the latest version of each api */
+    public static final short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1];
+
+    static {
+        REQUESTS[ApiKeys.PRODUCE.id] = PRODUCE_REQUEST;
+        REQUESTS[ApiKeys.FETCH.id] = FETCH_REQUEST;
+        REQUESTS[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_REQUEST;
+        REQUESTS[ApiKeys.METADATA.id] = METADATA_REQUEST;
+        REQUESTS[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
+        REQUESTS[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
+        REQUESTS[ApiKeys.UPDATE_METADATA_KEY.id] = new Schema[] {};
+        REQUESTS[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = new Schema[] {};
+        REQUESTS[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_REQUEST;
+        REQUESTS[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_REQUEST;
+        REQUESTS[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_REQUEST;
+        REQUESTS[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_REQUEST;
+        REQUESTS[ApiKeys.HEARTBEAT.id] = HEARTBEAT_REQUEST;
+
+        RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
+        RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE;
+        RESPONSES[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_RESPONSE;
+        RESPONSES[ApiKeys.METADATA.id] = METADATA_RESPONSE;
+        RESPONSES[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
+        RESPONSES[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
+        RESPONSES[ApiKeys.UPDATE_METADATA_KEY.id] = new Schema[] {};
+        RESPONSES[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = new Schema[] {};
+        RESPONSES[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_RESPONSE;
+        RESPONSES[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_RESPONSE;
+        RESPONSES[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_RESPONSE;
+        RESPONSES[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_RESPONSE;
+        RESPONSES[ApiKeys.HEARTBEAT.id] = HEARTBEAT_RESPONSE;
+
+        /* set the maximum version of each api */
+        for (ApiKeys api : ApiKeys.values())
+            CURR_VERSION[api.id] = (short) (REQUESTS[api.id].length - 1);
+
+        /* sanity check that we have the same number of request and response versions for each api */
+        for (ApiKeys api : ApiKeys.values())
+            if (REQUESTS[api.id].length != RESPONSES[api.id].length)
+                throw new IllegalStateException(REQUESTS[api.id].length + " request versions for api " + api.name
+                        + " but " + RESPONSES[api.id].length + " response versions.");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/SecurityProtocol.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/SecurityProtocol.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/SecurityProtocol.java
new file mode 100644
index 0000000..ab5a607
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/SecurityProtocol.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.kafka_backport.common.protocol;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public enum SecurityProtocol {
+    /** Un-authenticated, non-encrypted channel */
+    PLAINTEXT(0, "PLAINTEXT"),
+    /** Currently identical to PLAINTEXT and used for testing only. We may implement extra instrumentation when testing channel code. */
+    TRACE(Short.MAX_VALUE, "TRACE");
+
+    private static final Map<Short, SecurityProtocol> CODE_TO_SECURITY_PROTOCOL = new HashMap<Short, SecurityProtocol>();
+    private static final List<String> NAMES = new ArrayList<String>();
+
+    static {
+        for (SecurityProtocol proto: SecurityProtocol.values()) {
+            CODE_TO_SECURITY_PROTOCOL.put(proto.id, proto);
+            NAMES.add(proto.name);
+        }
+    }
+
+    /** The permanent and immutable id of a security protocol -- this can't change, and must match kafka.cluster.SecurityProtocol  */
+    public final short id;
+
+    /** Name of the security protocol. This may be used by client configuration. */
+    public final String name;
+
+    private SecurityProtocol(int id, String name) {
+        this.id = (short) id;
+        this.name = name;
+    }
+
+    public static String getName(int id) {
+        return CODE_TO_SECURITY_PROTOCOL.get((short) id).name;
+    }
+
+    public static List<String> getNames() {
+        return NAMES;
+    }
+
+    public static SecurityProtocol forId(Short id) {
+        return CODE_TO_SECURITY_PROTOCOL.get(id);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/ArrayOf.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/ArrayOf.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/ArrayOf.java
new file mode 100644
index 0000000..d2468d8
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/ArrayOf.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.protocol.types;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Represents a type for an array of a particular type
+ */
+public class ArrayOf extends Type {
+
+    private final Type type;
+
+    public ArrayOf(Type type) {
+        this.type = type;
+    }
+
+    @Override
+    public void write(ByteBuffer buffer, Object o) {
+        Object[] objs = (Object[]) o;
+        int size = objs.length;
+        buffer.putInt(size);
+        for (int i = 0; i < size; i++)
+            type.write(buffer, objs[i]);
+    }
+
+    @Override
+    public Object read(ByteBuffer buffer) {
+        int size = buffer.getInt();
+        Object[] objs = new Object[size];
+        for (int i = 0; i < size; i++)
+            objs[i] = type.read(buffer);
+        return objs;
+    }
+
+    @Override
+    public int sizeOf(Object o) {
+        Object[] objs = (Object[]) o;
+        int size = 4;
+        for (int i = 0; i < objs.length; i++)
+            size += type.sizeOf(objs[i]);
+        return size;
+    }
+
+    public Type type() {
+        return type;
+    }
+
+    @Override
+    public String toString() {
+        return "ARRAY(" + type + ")";
+    }
+
+    @Override
+    public Object[] validate(Object item) {
+        try {
+            Object[] array = (Object[]) item;
+            for (int i = 0; i < array.length; i++)
+                type.validate(array[i]);
+            return array;
+        } catch (ClassCastException e) {
+            throw new SchemaException("Not an Object[].");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Field.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Field.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Field.java
new file mode 100644
index 0000000..b7d7720
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Field.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.kafka_backport.common.protocol.types;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A field in a schema
+ */
+public class Field {
+
+    public static final Object NO_DEFAULT = new Object();
+
+    final int index;
+    public final String name;
+    public final Type type;
+    public final Object defaultValue;
+    public final String doc;
+    final Schema schema;
+
+    /**
+     * Create the field.
+     *
+     * @throws SchemaException If the default value is not primitive and the validation fails
+     */
+    public Field(int index, String name, Type type, String doc, Object defaultValue, Schema schema) {
+        this.index = index;
+        this.name = name;
+        this.type = type;
+        this.doc = doc;
+        this.defaultValue = defaultValue;
+        this.schema = schema;
+        if (defaultValue != NO_DEFAULT)
+            type.validate(defaultValue);
+    }
+
+    public Field(int index, String name, Type type, String doc, Object defaultValue) {
+        this(index, name, type, doc, defaultValue, null);
+    }
+
+    public Field(String name, Type type, String doc, Object defaultValue) {
+        this(-1, name, type, doc, defaultValue);
+    }
+
+    public Field(String name, Type type, String doc) {
+        this(name, type, doc, NO_DEFAULT);
+    }
+
+    public Field(String name, Type type) {
+        this(name, type, "");
+    }
+
+    public Type type() {
+        return type;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Schema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Schema.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Schema.java
new file mode 100644
index 0000000..7adac52
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Schema.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.kafka_backport.common.protocol.types;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The schema for a compound record definition
+ */
+public class Schema extends Type {
+
+    private final Field[] fields;
+    private final Map<String, Field> fieldsByName;
+
+    /**
+     * Construct the schema with a given list of its field values
+     *
+     * @throws SchemaException If the given list have duplicate fields
+     */
+    public Schema(Field... fs) {
+        this.fields = new Field[fs.length];
+        this.fieldsByName = new HashMap<String, Field>();
+        for (int i = 0; i < this.fields.length; i++) {
+            Field field = fs[i];
+            if (fieldsByName.containsKey(field.name))
+                throw new SchemaException("Schema contains a duplicate field: " + field.name);
+            this.fields[i] = new Field(i, field.name, field.type, field.doc, field.defaultValue, this);
+            this.fieldsByName.put(fs[i].name, this.fields[i]);
+        }
+    }
+
+    /**
+     * Write a struct to the buffer
+     */
+    public void write(ByteBuffer buffer, Object o) {
+        Struct r = (Struct) o;
+        for (int i = 0; i < fields.length; i++) {
+            Field f = fields[i];
+            try {
+                Object value = f.type().validate(r.get(f));
+                f.type.write(buffer, value);
+            } catch (Exception e) {
+                throw new SchemaException("Error writing field '" + f.name +
+                                          "': " +
+                                          (e.getMessage() == null ? e.getClass().getName() : e.getMessage()));
+            }
+        }
+    }
+
+    /**
+     * Read a struct from the buffer
+     */
+    public Object read(ByteBuffer buffer) {
+        Object[] objects = new Object[fields.length];
+        for (int i = 0; i < fields.length; i++) {
+            try {
+                objects[i] = fields[i].type.read(buffer);
+            } catch (Exception e) {
+                throw new SchemaException("Error reading field '" + fields[i].name +
+                                          "': " +
+                                          (e.getMessage() == null ? e.getClass().getName() : e.getMessage()));
+            }
+        }
+        return new Struct(this, objects);
+    }
+
+    /**
+     * The size of the given record
+     */
+    public int sizeOf(Object o) {
+        int size = 0;
+        Struct r = (Struct) o;
+        for (int i = 0; i < fields.length; i++)
+            size += fields[i].type.sizeOf(r.get(fields[i]));
+        return size;
+    }
+
+    /**
+     * The number of fields in this schema
+     */
+    public int numFields() {
+        return this.fields.length;
+    }
+
+    /**
+     * Get a field by its slot in the record array
+     * 
+     * @param slot The slot at which this field sits
+     * @return The field
+     */
+    public Field get(int slot) {
+        return this.fields[slot];
+    }
+
+    /**
+     * Get a field by its name
+     * 
+     * @param name The name of the field
+     * @return The field
+     */
+    public Field get(String name) {
+        return this.fieldsByName.get(name);
+    }
+
+    /**
+     * Get all the fields in this schema
+     */
+    public Field[] fields() {
+        return this.fields;
+    }
+
+    /**
+     * Display a string representation of the schema
+     */
+    public String toString() {
+        StringBuilder b = new StringBuilder();
+        b.append('{');
+        for (int i = 0; i < this.fields.length; i++) {
+            b.append(this.fields[i].name);
+            b.append(':');
+            b.append(this.fields[i].type());
+            if (i < this.fields.length - 1)
+                b.append(',');
+        }
+        b.append("}");
+        return b.toString();
+    }
+
+    @Override
+    public Struct validate(Object item) {
+        try {
+            Struct struct = (Struct) item;
+            for (int i = 0; i < this.fields.length; i++) {
+                Field field = this.fields[i];
+                try {
+                    field.type.validate(struct.get(field));
+                } catch (SchemaException e) {
+                    throw new SchemaException("Invalid value for field '" + field.name + "': " + e.getMessage());
+                }
+            }
+            return struct;
+        } catch (ClassCastException e) {
+            throw new SchemaException("Not a Struct.");
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/SchemaException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/SchemaException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/SchemaException.java
new file mode 100644
index 0000000..86c141e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/SchemaException.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.kafka_backport.common.protocol.types;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ *  Thrown if the protocol schema validation fails while parsing request or response.
+ */
+public class SchemaException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public SchemaException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Struct.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Struct.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Struct.java
new file mode 100644
index 0000000..482fe9d
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Struct.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.kafka_backport.common.protocol.types;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A record that can be serialized and deserialized according to a pre-defined schema
+ */
+public class Struct {
+    private final Schema schema;
+    private final Object[] values;
+
+    Struct(Schema schema, Object[] values) {
+        this.schema = schema;
+        this.values = values;
+    }
+
+    public Struct(Schema schema) {
+        this.schema = schema;
+        this.values = new Object[this.schema.numFields()];
+    }
+
+    /**
+     * The schema for this struct.
+     */
+    public Schema schema() {
+        return this.schema;
+    }
+
+    /**
+     * Return the value of the given pre-validated field, or if the value is missing return the default value.
+     * 
+     * @param field The field for which to get the default value
+     * @throws SchemaException if the field has no value and has no default.
+     */
+    private Object getFieldOrDefault(Field field) {
+        Object value = this.values[field.index];
+        if (value != null)
+            return value;
+        else if (field.defaultValue != Field.NO_DEFAULT)
+            return field.defaultValue;
+        else
+            throw new SchemaException("Missing value for field '" + field.name + "' which has no default value.");
+    }
+
+    /**
+     * Get the value for the field directly by the field index with no lookup needed (faster!)
+     * 
+     * @param field The field to look up
+     * @return The value for that field.
+     * @throws SchemaException if the field has no value and has no default.
+     */
+    public Object get(Field field) {
+        validateField(field);
+        return getFieldOrDefault(field);
+    }
+
+    /**
+     * Get the record value for the field with the given name by doing a hash table lookup (slower!)
+     * 
+     * @param name The name of the field
+     * @return The value in the record
+     * @throws SchemaException If no such field exists
+     */
+    public Object get(String name) {
+        Field field = schema.get(name);
+        if (field == null)
+            throw new SchemaException("No such field: " + name);
+        return getFieldOrDefault(field);
+    }
+
+    /**
+     * Check if the struct contains a field.
+     * @param name
+     * @return Whether a field exists.
+     */
+    public boolean hasField(String name) {
+        return schema.get(name) != null;
+    }
+
+    public Struct getStruct(Field field) {
+        return (Struct) get(field);
+    }
+
+    public Struct getStruct(String name) {
+        return (Struct) get(name);
+    }
+
+    public Short getShort(Field field) {
+        return (Short) get(field);
+    }
+
+    public Short getShort(String name) {
+        return (Short) get(name);
+    }
+
+    public Integer getInt(Field field) {
+        return (Integer) get(field);
+    }
+
+    public Integer getInt(String name) {
+        return (Integer) get(name);
+    }
+
+    public Long getLong(Field field) {
+        return (Long) get(field);
+    }
+
+    public Long getLong(String name) {
+        return (Long) get(name);
+    }
+
+    public Object[] getArray(Field field) {
+        return (Object[]) get(field);
+    }
+
+    public Object[] getArray(String name) {
+        return (Object[]) get(name);
+    }
+
+    public String getString(Field field) {
+        return (String) get(field);
+    }
+
+    public String getString(String name) {
+        return (String) get(name);
+    }
+
+    public ByteBuffer getBytes(Field field) {
+        return (ByteBuffer) get(field);
+    }
+
+    public ByteBuffer getBytes(String name) {
+        return (ByteBuffer) get(name);
+    }
+
+    /**
+     * Set the given field to the specified value
+     * 
+     * @param field The field
+     * @param value The value
+     * @throws SchemaException If the validation of the field failed
+     */
+    public Struct set(Field field, Object value) {
+        validateField(field);
+        this.values[field.index] = value;
+        return this;
+    }
+
+    /**
+     * Set the field specified by the given name to the value
+     * 
+     * @param name The name of the field
+     * @param value The value to set
+     * @throws SchemaException If the field is not known
+     */
+    public Struct set(String name, Object value) {
+        Field field = this.schema.get(name);
+        if (field == null)
+            throw new SchemaException("Unknown field: " + name);
+        this.values[field.index] = value;
+        return this;
+    }
+
+    /**
+     * Create a struct for the schema of a container type (struct or array). Note that for array type, this method
+     * assumes that the type is an array of schema and creates a struct of that schema. Arrays of other types can't be
+     * instantiated with this method.
+     * 
+     * @param field The field to create an instance of
+     * @return The struct
+     * @throws SchemaException If the given field is not a container type
+     */
+    public Struct instance(Field field) {
+        validateField(field);
+        if (field.type() instanceof Schema) {
+            return new Struct((Schema) field.type());
+        } else if (field.type() instanceof ArrayOf) {
+            ArrayOf array = (ArrayOf) field.type();
+            return new Struct((Schema) array.type());
+        } else {
+            throw new SchemaException("Field '" + field.name + "' is not a container type, it is of type " + field.type());
+        }
+    }
+
+    /**
+     * Create a struct instance for the given field which must be a container type (struct or array)
+     * 
+     * @param field The name of the field to create (field must be a schema type)
+     * @return The struct
+     * @throws SchemaException If the given field is not a container type
+     */
+    public Struct instance(String field) {
+        return instance(schema.get(field));
+    }
+
+    /**
+     * Empty all the values from this record
+     */
+    public void clear() {
+        Arrays.fill(this.values, null);
+    }
+
+    /**
+     * Get the serialized size of this object
+     */
+    public int sizeOf() {
+        return this.schema.sizeOf(this);
+    }
+
+    /**
+     * Write this struct to a buffer
+     */
+    public void writeTo(ByteBuffer buffer) {
+        this.schema.write(buffer, this);
+    }
+
+    /**
+     * Ensure the user doesn't try to access fields from the wrong schema
+     *
+     * @throws SchemaException If validation fails
+     */
+    private void validateField(Field field) {
+        if (this.schema != field.schema)
+            throw new SchemaException("Attempt to access field '" + field.name + "' from a different schema instance.");
+        if (field.index > values.length)
+            throw new SchemaException("Invalid field index: " + field.index);
+    }
+
+    /**
+     * Validate the contents of this struct against its schema
+     *
+     * @throws SchemaException If validation fails
+     */
+    public void validate() {
+        this.schema.validate(this);
+    }
+
+    /**
+     * Create a byte buffer containing the serialized form of the values in this struct. This method can choose to break
+     * the struct into multiple ByteBuffers if need be.
+     */
+    public ByteBuffer[] toBytes() {
+        ByteBuffer buffer = ByteBuffer.allocate(sizeOf());
+        writeTo(buffer);
+        return new ByteBuffer[] {buffer};
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder b = new StringBuilder();
+        b.append('{');
+        for (int i = 0; i < this.values.length; i++) {
+            Field f = this.schema.get(i);
+            b.append(f.name);
+            b.append('=');
+            if (f.type() instanceof ArrayOf) {
+                Object[] arrayValue = (Object[]) this.values[i];
+                b.append('[');
+                for (int j = 0; j < arrayValue.length; j++) {
+                    b.append(arrayValue[j]);
+                    if (j < arrayValue.length - 1)
+                        b.append(',');
+                }
+                b.append(']');
+            } else
+                b.append(this.values[i]);
+            if (i < this.values.length - 1)
+                b.append(',');
+        }
+        b.append('}');
+        return b.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        for (int i = 0; i < this.values.length; i++) {
+            Field f = this.schema.get(i);
+            if (f.type() instanceof ArrayOf) {
+                Object[] arrayObject = (Object[]) this.get(f);
+                for (Object arrayItem: arrayObject)
+                    result = prime * result + arrayItem.hashCode();
+            } else {
+                result = prime * result + this.get(f).hashCode();
+            }
+        }
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        Struct other = (Struct) obj;
+        if (schema != other.schema)
+            return false;
+        for (int i = 0; i < this.values.length; i++) {
+            Field f = this.schema.get(i);
+            Boolean result;
+            if (f.type() instanceof ArrayOf) {
+                result = Arrays.equals((Object[]) this.get(f), (Object[]) other.get(f));
+            } else {
+                result = this.get(f).equals(other.get(f));
+            }
+            if (!result)
+                return false;
+        }
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Type.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Type.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Type.java
new file mode 100644
index 0000000..26bdd2f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Type.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.kafka_backport.common.protocol.types;
+
+import org.apache.flink.kafka_backport.common.utils.Utils;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A serializable type
+ */
+public abstract class Type {
+
+    /**
+     * Write the typed object to the buffer
+     *
+     * @throws SchemaException If the object is not valid for its type
+     */
+    public abstract void write(ByteBuffer buffer, Object o);
+
+    /**
+     * Read the typed object from the buffer
+     *
+     * @throws SchemaException If the object is not valid for its type
+     */
+    public abstract Object read(ByteBuffer buffer);
+
+    /**
+     * Validate the object. If succeeded return its typed object.
+     *
+     * @throws SchemaException If validation failed
+     */
+    public abstract Object validate(Object o);
+
+    /**
+     * Return the size of the object in bytes
+     */
+    public abstract int sizeOf(Object o);
+
+    public static final Type INT8 = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            buffer.put((Byte) o);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            return buffer.get();
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 1;
+        }
+
+        @Override
+        public String toString() {
+            return "INT8";
+        }
+
+        @Override
+        public Byte validate(Object item) {
+            if (item instanceof Byte)
+                return (Byte) item;
+            else
+                throw new SchemaException(item + " is not a Byte.");
+        }
+    };
+
+    public static final Type INT16 = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            buffer.putShort((Short) o);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            return buffer.getShort();
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 2;
+        }
+
+        @Override
+        public String toString() {
+            return "INT16";
+        }
+
+        @Override
+        public Short validate(Object item) {
+            if (item instanceof Short)
+                return (Short) item;
+            else
+                throw new SchemaException(item + " is not a Short.");
+        }
+    };
+
+    public static final Type INT32 = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            buffer.putInt((Integer) o);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            return buffer.getInt();
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 4;
+        }
+
+        @Override
+        public String toString() {
+            return "INT32";
+        }
+
+        @Override
+        public Integer validate(Object item) {
+            if (item instanceof Integer)
+                return (Integer) item;
+            else
+                throw new SchemaException(item + " is not an Integer.");
+        }
+    };
+
+    public static final Type INT64 = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            buffer.putLong((Long) o);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            return buffer.getLong();
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 8;
+        }
+
+        @Override
+        public String toString() {
+            return "INT64";
+        }
+
+        @Override
+        public Long validate(Object item) {
+            if (item instanceof Long)
+                return (Long) item;
+            else
+                throw new SchemaException(item + " is not a Long.");
+        }
+    };
+
+    public static final Type STRING = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            byte[] bytes = Utils.utf8((String) o);
+            if (bytes.length > Short.MAX_VALUE)
+                throw new SchemaException("String is longer than the maximum string length.");
+            buffer.putShort((short) bytes.length);
+            buffer.put(bytes);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            int length = buffer.getShort();
+            byte[] bytes = new byte[length];
+            buffer.get(bytes);
+            return Utils.utf8(bytes);
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 2 + Utils.utf8Length((String) o);
+        }
+
+        @Override
+        public String toString() {
+            return "STRING";
+        }
+
+        @Override
+        public String validate(Object item) {
+            if (item instanceof String)
+                return (String) item;
+            else
+                throw new SchemaException(item + " is not a String.");
+        }
+    };
+
+    public static final Type BYTES = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            ByteBuffer arg = (ByteBuffer) o;
+            int pos = arg.position();
+            buffer.putInt(arg.remaining());
+            buffer.put(arg);
+            arg.position(pos);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            int size = buffer.getInt();
+            ByteBuffer val = buffer.slice();
+            val.limit(size);
+            buffer.position(buffer.position() + size);
+            return val;
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            ByteBuffer buffer = (ByteBuffer) o;
+            return 4 + buffer.remaining();
+        }
+
+        @Override
+        public String toString() {
+            return "BYTES";
+        }
+
+        @Override
+        public ByteBuffer validate(Object item) {
+            if (item instanceof ByteBuffer)
+                return (ByteBuffer) item;
+            else
+                throw new SchemaException(item + " is not a java.nio.ByteBuffer.");
+        }
+    };
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferInputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferInputStream.java
new file mode 100644
index 0000000..99a20a3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferInputStream.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.kafka_backport.common.record;
+
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A byte buffer backed input outputStream
+ */
+public class ByteBufferInputStream extends InputStream {
+
+    private ByteBuffer buffer;
+
+    public ByteBufferInputStream(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    public int read() {
+        if (!buffer.hasRemaining()) {
+            return -1;
+        }
+        return buffer.get() & 0xFF;
+    }
+
+    public int read(byte[] bytes, int off, int len) {
+        if (!buffer.hasRemaining()) {
+            return -1;
+        }
+
+        len = Math.min(len, buffer.remaining());
+        buffer.get(bytes, off, len);
+        return len;
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferOutputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferOutputStream.java
new file mode 100644
index 0000000..a334755
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferOutputStream.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.kafka_backport.common.record;
+
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A byte buffer backed output outputStream
+ */
+public class ByteBufferOutputStream extends OutputStream {
+
+    private static final float REALLOCATION_FACTOR = 1.1f;
+
+    private ByteBuffer buffer;
+
+    public ByteBufferOutputStream(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    public void write(int b) {
+        if (buffer.remaining() < 1)
+            expandBuffer(buffer.capacity() + 1);
+        buffer.put((byte) b);
+    }
+
+    public void write(byte[] bytes, int off, int len) {
+        if (buffer.remaining() < len)
+            expandBuffer(buffer.capacity() + len);
+        buffer.put(bytes, off, len);
+    }
+
+    public ByteBuffer buffer() {
+        return buffer;
+    }
+
+    private void expandBuffer(int size) {
+        int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
+        ByteBuffer temp = ByteBuffer.allocate(expandSize);
+        temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
+        buffer = temp;
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/CompressionType.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/CompressionType.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/CompressionType.java
new file mode 100644
index 0000000..9961766
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/CompressionType.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.kafka_backport.common.record;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The compression type to use
+ */
+public enum CompressionType {
+    NONE(0, "none", 1.0f), GZIP(1, "gzip", 0.5f), SNAPPY(2, "snappy", 0.5f), LZ4(3, "lz4", 0.5f);
+
+    public final int id;
+    public final String name;
+    public final float rate;
+
+    private CompressionType(int id, String name, float rate) {
+        this.id = id;
+        this.name = name;
+        this.rate = rate;
+    }
+
+    public static CompressionType forId(int id) {
+        switch (id) {
+            case 0:
+                return NONE;
+            case 1:
+                return GZIP;
+            case 2:
+                return SNAPPY;
+            case 3:
+                return LZ4;
+            default:
+                throw new IllegalArgumentException("Unknown compression type id: " + id);
+        }
+    }
+
+    public static CompressionType forName(String name) {
+        if (NONE.name.equals(name))
+            return NONE;
+        else if (GZIP.name.equals(name))
+            return GZIP;
+        else if (SNAPPY.name.equals(name))
+            return SNAPPY;
+        else if (LZ4.name.equals(name))
+            return LZ4;
+        else
+            throw new IllegalArgumentException("Unknown compression name: " + name);
+    }
+
+}


[29/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Refactor, cleanup, and fix kafka consumers

Posted by se...@apache.org.
[FLINK-2386] [kafka connector] Refactor, cleanup, and fix kafka consumers


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

Branch: refs/heads/master
Commit: b9892a0ecfbd1feb4ab75aa02d4c5ca015fef508
Parents: 33f4c81
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Aug 11 22:21:53 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Aug 27 12:40:38 2015 +0200

----------------------------------------------------------------------
 .../flink-connector-kafka-083/pom.xml           |    8 +
 .../flink/kafka_backport/package-info.java      |   19 +-
 .../connectors/FlinkKafkaConsumer.java          |  615 +++++++++
 .../connectors/FlinkKafkaConsumer081.java       |   36 +-
 .../connectors/FlinkKafkaConsumer082.java       |   28 +-
 .../connectors/FlinkKafkaConsumer083.java       |   27 +-
 .../streaming/connectors/internals/Fetcher.java |   71 +-
 .../internals/FlinkKafkaConsumerBase.java       |  429 ------
 .../connectors/internals/IncludedFetcher.java   |  105 --
 .../connectors/internals/LegacyFetcher.java     |  625 ++++++---
 .../internals/NewConsumerApiFetcher.java        |  123 ++
 .../connectors/internals/OffsetHandler.java     |   57 +
 .../internals/ZooKeeperStringSerializer.java    |   51 +
 .../internals/ZookeeperOffsetHandler.java       |  128 ++
 .../streaming/connectors/Kafka081ITCase.java    |  101 +-
 .../streaming/connectors/Kafka082ITCase.java    |  104 +-
 .../streaming/connectors/Kafka083ITCase.java    |   84 --
 .../KafkaConsumerPartitionAssignmentTest.java   |  256 ++++
 .../streaming/connectors/KafkaConsumerTest.java |  143 ++
 .../connectors/KafkaConsumerTestBase.java       | 1137 ++++++++++++++++
 .../connectors/KafkaLocalSystemTime.java        |    1 +
 .../connectors/KafkaProducerITCase.java         |  186 +++
 .../streaming/connectors/KafkaTestBase.java     | 1279 +++---------------
 .../internals/ZookeeperOffsetHandlerTest.java   |   59 +
 .../connectors/testutils/DataGenerators.java    |  211 +++
 .../connectors/testutils/DiscardingSink.java    |   33 +
 .../testutils/FailingIdentityMapper.java        |  115 ++
 .../testutils/JobManagerCommunicationUtils.java |   76 ++
 .../testutils/MockRuntimeContext.java           |  131 ++
 .../testutils/PartitionValidatingMapper.java    |   53 +
 .../connectors/testutils/SuccessException.java  |   26 +
 .../connectors/testutils/ThrottledMapper.java   |   44 +
 .../connectors/testutils/Tuple2Partitioner.java |   49 +
 .../testutils/ValidatingExactlyOnceSink.java    |   76 ++
 .../src/test/resources/log4j-test.properties    |    6 +-
 .../streaming/connectors/kafka/KafkaITCase.java |   58 +-
 .../connectors/kafka/util/UtilsTest.java        |   75 -
 .../streaming/runtime/io/BarrierBuffer.java     |   44 +-
 .../TypeInformationSerializationSchema.java     |   11 -
 .../TypeInformationSerializationSchemaTest.java |   16 +-
 40 files changed, 4556 insertions(+), 2140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/pom.xml
index 8aa1bff..0996e8f2 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/pom.xml
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/pom.xml
@@ -96,6 +96,14 @@ under the License.
 			<groupId>org.apache.curator</groupId>
 			<artifactId>curator-test</artifactId>
 			<version>2.7.1</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
 		</dependency>
 
 	</dependencies>

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
index 5852c13..6a574c8 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 /**
  *  This package (and its sub-packages) contain only classes that arecopied from
  *  the Apache Kafka project.
@@ -7,4 +24,4 @@
  *
  *  This is a temporary workaround!
  */
-package org.apache.flink.kafka_backport;
\ No newline at end of file
+package org.apache.flink.kafka_backport;

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

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer081.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer081.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer081.java
index 27e056a..f696893 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer081.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer081.java
@@ -14,18 +14,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.streaming.connectors;
 
-import org.apache.flink.streaming.connectors.internals.FlinkKafkaConsumerBase;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import java.util.Properties;
 
-public class FlinkKafkaConsumer081<T> extends FlinkKafkaConsumerBase<T> {
+/**
+ * Creates a Kafka consumer compatible with reading from Kafka 0.8.1.x brokers.
+ * The consumer will internally use the old low-level Kafka API, and manually commit offsets
+ * partition offsets to ZooKeeper.
+ * 
+ * <p>The following additional configuration values are available:</p>
+ * <ul>
+ *   <li>socket.timeout.ms</li>
+ *   <li>socket.receive.buffer.bytes</li>
+ *   <li>fetch.message.max.bytes</li>
+ *   <li>auto.offset.reset with the values "latest", "earliest" (unlike 0.8.2 behavior)</li>
+ *   <li>fetch.wait.max.ms</li>
+ * </ul>
+ * 
+ * @param <T> The type of elements produced by this consumer.
+ */
+public class FlinkKafkaConsumer081<T> extends FlinkKafkaConsumer<T> {
+
+	private static final long serialVersionUID = -5649906773771949146L;
 
+	/**
+	 * Creates a new Kafka 0.8.1.x streaming source consumer.
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects. 
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
 	public FlinkKafkaConsumer081(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		super(topic, valueDeserializer, props);
-		this.offsetStore = OffsetStore.FLINK_ZOOKEEPER;
-		this.fetcherType = FetcherType.LEGACY;
+		super(topic, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer082.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer082.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer082.java
index b775db5..1540f8f 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer082.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer082.java
@@ -14,25 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.streaming.connectors;
 
-import org.apache.flink.streaming.connectors.internals.FlinkKafkaConsumerBase;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import java.util.Properties;
 
 /**
- * Kafka consumer for Kafka 0.8.2.x
- *
- * It commits the offsets to Zookeeper (Flink code) and uses the consumer of Kafka 0.8.3 (currently WIP).
+ * Creates a Kafka consumer compatible with reading from Kafka 0.8.2.x brokers.
+ * The consumer will use the new Kafka consumer API (early Flink backport version),
+ * and manually commit offsets partition offsets to ZooKeeper.
  *
- * @param <T>
+ * @param <T> The type of elements produced by this consumer.
  */
-public class FlinkKafkaConsumer082<T> extends FlinkKafkaConsumerBase<T> {
+public class FlinkKafkaConsumer082<T> extends FlinkKafkaConsumer<T> {
+
+	private static final long serialVersionUID = -8450689820627198228L;
 
+	/**
+	 * Creates a new Kafka 0.8.2.x streaming source consumer.
+	 * 
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects. 
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
 	public FlinkKafkaConsumer082(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		super(topic, valueDeserializer, props);
-		this.offsetStore = OffsetStore.FLINK_ZOOKEEPER;
-		this.fetcherType = FetcherType.INCLUDED;
+		super(topic, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.NEW_HIGH_LEVEL);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer083.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer083.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer083.java
index 54563db..6ed2930 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer083.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer083.java
@@ -14,18 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.streaming.connectors;
 
-import org.apache.flink.streaming.connectors.internals.FlinkKafkaConsumerBase;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import java.util.Properties;
 
-public class FlinkKafkaConsumer083<T> extends FlinkKafkaConsumerBase<T> {
+/**
+ * Creates a Kafka consumer compatible with reading from Kafka 0.8.3.x brokers.
+ * The consumer will use the new Kafka consumer API (early Flink backport version),
+ * and lets Kafka handle the offset committing internally.
+ * 
+ * @param <T> The type of elements produced by this consumer.
+ */
+public class FlinkKafkaConsumer083<T> extends FlinkKafkaConsumer<T> {
+
+	private static final long serialVersionUID = 1126432820518992927L;
 
+	/**
+	 * Creates a new Kafka 0.8.3.x streaming source consumer.
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects. 
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
 	public FlinkKafkaConsumer083(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		super(topic, valueDeserializer, props);
-		this.offsetStore = OffsetStore.KAFKA;
-		this.fetcherType = FetcherType.INCLUDED;
+		super(topic, valueDeserializer, props, OffsetStore.KAFKA, FetcherType.NEW_HIGH_LEVEL);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java
index 318b2f8..25a3fea 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java
@@ -14,51 +14,70 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.streaming.connectors.internals;
 
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.kafka_backport.common.TopicPartition;
 
+import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
+/**
+ * A fetcher pulls data from Kafka, from a fix set of partitions.
+ * The fetcher supports "seeking" inside the partitions, i.e., moving to a different offset.
+ */
 public interface Fetcher {
 
 	/**
-	 * Set which partitions we want to read from
-	 * @param partitions
-	 */
-	void partitionsToRead(List<TopicPartition> partitions);
-
-	/**
-	 * Ask the run() method to stop reading
+	 * Set which partitions the fetcher should pull from.
+	 * 
+	 * @param partitions The list of partitions for a topic that the fetcher will pull from.
 	 */
-	void stop();
+	void setPartitionsToRead(List<TopicPartition> partitions);
 
 	/**
-	 * Close the underlying connection
+	 * Closes the fetcher. This will stop any operation in the
+	 * {@link #run(SourceFunction.SourceContext, DeserializationSchema, long[])} method and eventually
+	 * close underlying connections and release all resources.
 	 */
-	void close();
+	void close() throws IOException;
 
 	/**
-	 * Start and fetch indefinitely from the underlying fetcher
-	 * @param sourceContext
-	 * @param valueDeserializer
-	 * @param lastOffsets
-	 * @param <T>
+	 * Starts fetch data from Kafka and emitting it into the stream.
+	 * 
+	 * <p>To provide exactly once guarantees, the fetcher needs emit a record and update the update
+	 * of the last consumed offset in one atomic operation:</p>
+	 * <pre>{@code
+	 * 
+	 * while (running) {
+	 *     T next = ...
+	 *     long offset = ...
+	 *     int partition = ...
+	 *     synchronized (sourceContext.getCheckpointLock()) {
+	 *         sourceContext.collect(next);
+	 *         lastOffsets[partition] = offset;
+	 *     }
+	 * }
+	 * }</pre>
+	 * 
+	 * @param sourceContext The source context to emit elements to.
+	 * @param valueDeserializer The deserializer to decode the raw values with.
+	 * @param lastOffsets The array into which to store the offsets foe which elements are emitted. 
+	 * 
+	 * @param <T> The type of elements produced by the fetcher and emitted to the source context.
 	 */
-	<T> void run(SourceFunction.SourceContext<T> sourceContext, DeserializationSchema<T> valueDeserializer, long[] lastOffsets);
-
-	/**
-	 * Commit offset (if supported)
-	 * @param offsetsToCommit
-	 */
-	void commit(Map<TopicPartition, Long> offsetsToCommit);
-
+	<T> void run(SourceFunction.SourceContext<T> sourceContext, DeserializationSchema<T> valueDeserializer, 
+					long[] lastOffsets) throws Exception;
+	
 	/**
-	 * Set offsets for the partitions.
-	 * The offset is the next offset to read. So if set to 0, the Fetcher's first result will be the msg with offset=0.
+	 * Set the next offset to read from for the given partition.
+	 * For example, if the partition <i>i</i> offset is set to <i>n</i>, the Fetcher's next result
+	 * will be the message with <i>offset=n</i>.
+	 * 
+	 * @param topicPartition The partition for which to seek the offset.
+	 * @param offsetToRead To offset to seek to.
 	 */
 	void seek(TopicPartition topicPartition, long offsetToRead);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/FlinkKafkaConsumerBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/FlinkKafkaConsumerBase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/FlinkKafkaConsumerBase.java
deleted file mode 100644
index 7450410..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/FlinkKafkaConsumerBase.java
+++ /dev/null
@@ -1,429 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.internals;
-
-import kafka.common.TopicAndPartition;
-import kafka.utils.ZKGroupTopicDirs;
-import kafka.utils.ZkUtils;
-import org.I0Itec.zkclient.ZkClient;
-import org.I0Itec.zkclient.exception.ZkMarshallingError;
-import org.I0Itec.zkclient.serialize.ZkSerializer;
-import org.apache.commons.collections.map.LinkedMap;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.kafka_backport.clients.consumer.ConsumerConfig;
-import org.apache.flink.kafka_backport.clients.consumer.KafkaConsumer;
-import org.apache.flink.kafka_backport.common.PartitionInfo;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.serialization.ByteArrayDeserializer;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Option;
-
-import java.io.UnsupportedEncodingException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-/**
- * When using the legacy fetcher, the following additional configuration values are available:
- *
- * - socket.timeout.ms
- * - socket.receive.buffer.bytes
- * - fetch.message.max.bytes
- * - auto.offset.reset with the values "latest", "earliest" (unlike 0.8.2 behavior)
- * - flink.kafka.consumer.queue.size (Size of the queue between the fetching threads)
- * - fetch.wait.max.ms
- */
-public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFunction<T>
-		implements CheckpointNotifier, CheckpointedAsynchronously<long[]>, ResultTypeQueryable {
-
-	public static Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumerBase.class);
-
-	static final long OFFSET_NOT_SET = -1L;
-
-
-	private final String topic;
-	private final Properties props;
-	private final int[] partitions; // ordered, immutable partition assignment list
-	private final DeserializationSchema<T> valueDeserializer;
-
-	private transient Fetcher fetcher;
-	private final LinkedMap pendingCheckpoints = new LinkedMap();
-	private long[] lastOffsets;
-	protected long[] commitedOffsets;
-	private ZkClient zkClient;
-	private long[] restoreToOffset;
-	protected OffsetStore offsetStore = OffsetStore.FLINK_ZOOKEEPER;
-	protected FetcherType fetcherType = FetcherType.LEGACY;
-	private boolean isNoOp = false; // no-operation instance (for example when there are fewer partitions that flink consumers)
-	private boolean closed = false;
-
-	public enum OffsetStore {
-		FLINK_ZOOKEEPER,
-		/*
-		Let Flink manage the offsets. It will store them in Zookeeper, in the same structure as Kafka 0.8.2.x
-		Use this mode when using the source with Kafka 0.8.x brokers
-		*/
-		KAFKA
-		/*
-		Use the mechanisms in Kafka to commit offsets. Depending on the Kafka configuration, different mechanism are used (broker coordinator, zookeeper)
-		*/
-	}
-
-	public enum FetcherType {
-		LEGACY,  /* Use this fetcher for Kafka 0.8.1 brokers */
-		INCLUDED /* This fetcher works with Kafka 0.8.2 and 0.8.3 */
-	}
-
-
-	public FlinkKafkaConsumerBase(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		this.topic = topic;
-		this.props = props; // TODO check for zookeeper properties
-		this.valueDeserializer = valueDeserializer;
-
-		// Connect to a broker to get the partitions
-		List<PartitionInfo> partitionInfos = getPartitionsForTopic(this.topic, this.props);
-
-		// get initial partitions list. The order of the partitions is important for consistent partition id assignment
-		// in restart cases.
-		// Note that the source will fail (= job will restart) in case of a broker failure
-		partitions = new int[partitionInfos.size()];
-		for(int i = 0; i < partitionInfos.size(); i++) {
-			partitions[i] = partitionInfos.get(i).partition();
-		}
-		LOG.info("Topic {} has {} partitions", topic, partitions.length);
-	}
-
-	// ----------------------------- Source ------------------------------
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-
-		// make sure that we take care of the committing
-		props.setProperty("enable.auto.commit", "false");
-
-		// create fetcher
-		switch(fetcherType){
-			case INCLUDED:
-				fetcher = new IncludedFetcher(props);
-				break;
-			case LEGACY:
-				fetcher = new LegacyFetcher(topic, props);
-				break;
-			default:
-				throw new RuntimeException("Requested unknown fetcher "+fetcher);
-		}
-
-		// tell which partitions we want to subscribe
-		List<TopicPartition> partitionsToSub = assignPartitions(this.partitions);
-		LOG.info("This instance (id={}) is going to subscribe to partitions {}", getRuntimeContext().getIndexOfThisSubtask(), partitionsToSub);
-		if(partitionsToSub.size() == 0) {
-			LOG.info("This instance is a no-op instance.");
-			isNoOp = true;
-			return;
-		}
-		fetcher.partitionsToRead(partitionsToSub);
-
-		// set up operator state
-		lastOffsets = new long[partitions.length];
-		Arrays.fill(lastOffsets, OFFSET_NOT_SET);
-
-		// prepare Zookeeper
-		if(offsetStore == OffsetStore.FLINK_ZOOKEEPER) {
-			String zkConnect = props.getProperty("zookeeper.connect");
-			if(zkConnect == null) {
-				throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set");
-			}
-			zkClient = new ZkClient(zkConnect,
-					Integer.valueOf(props.getProperty("zookeeper.session.timeout.ms", "6000")),
-					Integer.valueOf(props.getProperty("zookeeper.connection.timeout.ms", "6000")),
-					new KafkaZKStringSerializer());
-		}
-		commitedOffsets = new long[partitions.length];
-
-
-		// seek to last known pos, from restore request
-		if(restoreToOffset != null) {
-			LOG.info("Found offsets to restore to: "+Arrays.toString(restoreToOffset));
-			for(int i = 0; i < restoreToOffset.length; i++) {
-				if(restoreToOffset[i] != OFFSET_NOT_SET) {
-					// if this fails because we are not subscribed to the topic, the partition assignment is not deterministic!
-					// we set the offset +1 here, because seek() is accepting the next offset to read, but the restore offset is the last read offset
-					fetcher.seek(new TopicPartition(topic, i), restoreToOffset[i] + 1);
-				}
-			}
-		} else {
-			// no restore request. See what we have in ZK for this consumer group. In the non ZK case, Kafka will take care of this.
-			if(offsetStore == OffsetStore.FLINK_ZOOKEEPER) {
-				for (TopicPartition tp : partitionsToSub) {
-					long offset = getOffset(zkClient, props.getProperty(ConsumerConfig.GROUP_ID_CONFIG), topic, tp.partition());
-					if (offset != OFFSET_NOT_SET) {
-						LOG.info("Offset for partition {} was set to {} in ZK. Seeking consumer to that position", tp.partition(), offset);
-						// the offset in Zookeeper was the last read offset, seek is accepting the next-to-read-offset.
-						fetcher.seek(tp, offset + 1);
-					}
-				}
-			}
-		}
-
-	}
-
-	@Override
-	public void run(SourceContext<T> sourceContext) throws Exception {
-		if(isNoOp) {
-			sourceContext.close();
-			return;
-		}
-		fetcher.run(sourceContext, valueDeserializer, lastOffsets);
-	}
-
-	@Override
-	public void cancel() {
-		if(isNoOp) {
-			return;
-		}
-		fetcher.stop();
-		fetcher.close();
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		closed = true;
-	}
-
-	@Override
-	public TypeInformation getProducedType() {
-		return valueDeserializer.getProducedType();
-	}
-
-
-	public List<TopicPartition> assignPartitions(int[] parts) {
-		LOG.info("Assigning partitions from "+Arrays.toString(parts));
-		List<TopicPartition> partitionsToSub = new ArrayList<TopicPartition>();
-
-		int machine = 0;
-		for(int i = 0; i < parts.length; i++) {
-			if(machine == getRuntimeContext().getIndexOfThisSubtask()) {
-				partitionsToSub.add(new TopicPartition(topic, parts[i]));
-			}
-			machine++;
-
-			if(machine == getRuntimeContext().getNumberOfParallelSubtasks()) {
-				machine = 0;
-			}
-		}
-
-		return partitionsToSub;
-	}
-
-
-	// ----------------------------- Utilities -------------------------
-
-	/**
-	 * Send request to Kafka cluster to get partitions for topic.
-	 */
-	protected static List<PartitionInfo> getPartitionsForTopic(String topic, Properties properties) {
-		KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<byte[], byte[]>(properties, null,
-				new ByteArrayDeserializer(), new ByteArrayDeserializer());
-
-		try {
-			List<PartitionInfo> partitions = consumer.partitionsFor(topic);
-			if (partitions == null) {
-				throw new RuntimeException("The topic " + topic + " does not seem to exist");
-			}
-			if(partitions.size() == 0) {
-				throw new RuntimeException("The topic "+topic+" does not seem to have any partitions");
-			}
-			return partitions;
-		} finally {
-			consumer.close();
-		}
-	}
-
-	// ----------------------------- State ------------------------------
-
-	@Override
-	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		if(fetcher == null) {
-			LOG.info("notifyCheckpointComplete() called on uninitialized source");
-			return;
-		}
-		if(closed) {
-			LOG.info("notifyCheckpointComplete() called on closed source");
-			return;
-		}
-
-		LOG.info("Commit checkpoint {}", checkpointId);
-
-		long[] checkpointOffsets;
-
-		// the map may be asynchronously updates when snapshotting state, so we synchronize
-		synchronized (pendingCheckpoints) {
-			final int posInMap = pendingCheckpoints.indexOf(checkpointId);
-			if (posInMap == -1) {
-				LOG.warn("Unable to find pending checkpoint for id {}", checkpointId);
-				return;
-			}
-
-			checkpointOffsets = (long[]) pendingCheckpoints.remove(posInMap);
-			// remove older checkpoints in map:
-			if (!pendingCheckpoints.isEmpty()) {
-				for(int i = 0; i < posInMap; i++) {
-					pendingCheckpoints.remove(0);
-				}
-			}
-		}
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Committing offsets {} to offset store: {}", Arrays.toString(checkpointOffsets), offsetStore);
-		}
-
-		if(offsetStore == OffsetStore.FLINK_ZOOKEEPER) {
-			setOffsetsInZooKeeper(checkpointOffsets);
-		} else {
-			Map<TopicPartition, Long> offsetsToCommit = new HashMap<TopicPartition, Long>();
-			for(int i = 0; i < checkpointOffsets.length; i++) {
-				if(checkpointOffsets[i] != OFFSET_NOT_SET) {
-					offsetsToCommit.put(new TopicPartition(topic, i), checkpointOffsets[i]);
-				}
-			}
-			fetcher.commit(offsetsToCommit);
-		}
-	}
-
-	@Override
-	public long[] snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-		if (lastOffsets == null) {
-			LOG.warn("State snapshot requested on not yet opened source. Returning null");
-			return null;
-		}
-		if(closed) {
-			LOG.info("snapshotState() called on closed source");
-			return null;
-		}
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Snapshotting state. Offsets: {}, checkpoint id {}, timestamp {}",
-					Arrays.toString(lastOffsets), checkpointId, checkpointTimestamp);
-		}
-
-		long[] currentOffsets = Arrays.copyOf(lastOffsets, lastOffsets.length);
-
-		// the map may be asynchronously updated when committing to Kafka, so we synchronize
-		synchronized (pendingCheckpoints) {
-			pendingCheckpoints.put(checkpointId, currentOffsets);
-		}
-
-		return currentOffsets;
-	}
-
-	@Override
-	public void restoreState(long[] restoredOffsets) {
-		restoreToOffset = restoredOffsets;
-	}
-
-	// ---------- Zookeeper communication ----------------
-
-	private void setOffsetsInZooKeeper(long[] offsets) {
-		for (int partition = 0; partition < offsets.length; partition++) {
-			long offset = offsets[partition];
-			if(offset != OFFSET_NOT_SET) {
-				setOffset(partition, offset);
-			}
-		}
-	}
-
-	protected void setOffset(int partition, long offset) {
-		// synchronize because notifyCheckpointComplete is called using asynchronous worker threads (= multiple checkpoints might be confirmed concurrently)
-		synchronized (commitedOffsets) {
-			if(closed) {
-				// it might happen that the source has been closed while the asynchronous commit threads waited for committing the offsets.
-				LOG.warn("setOffset called on closed source");
-				return;
-			}
-			if(commitedOffsets[partition] < offset) {
-				setOffset(zkClient, props.getProperty(ConsumerConfig.GROUP_ID_CONFIG), topic, partition, offset);
-				commitedOffsets[partition] = offset;
-			} else {
-				LOG.debug("Ignoring offset {} for partition {} because it is already committed", offset, partition);
-			}
-		}
-	}
-
-	// the following two methods are static to allow access from the outside as well (Testcases)
-
-	/**
-	 * This method's code is based on ZookeeperConsumerConnector.commitOffsetToZooKeeper()
-	 */
-	public static void setOffset(ZkClient zkClient, String groupId, String topic, int partition, long offset) {
-		LOG.info("Setting offset for partition {} of topic {} in group {} to offset {}", partition, topic, groupId, offset);
-		TopicAndPartition tap = new TopicAndPartition(topic, partition);
-		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, tap.topic());
-		ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir() + "/" + tap.partition(), Long.toString(offset));
-	}
-
-	public static long getOffset(ZkClient zkClient, String groupId, String topic, int partition) {
-		TopicAndPartition tap = new TopicAndPartition(topic, partition);
-		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, tap.topic());
-		scala.Tuple2<Option<String>, Stat> data = ZkUtils.readDataMaybeNull(zkClient, topicDirs.consumerOffsetDir() + "/" + tap.partition());
-		if(data._1().isEmpty()) {
-			return OFFSET_NOT_SET;
-		} else {
-			return Long.valueOf(data._1().get());
-		}
-	}
-
-	// ---------------------- Zookeeper Serializer copied from Kafka (because it has private access there)  -----------------
-	public static class KafkaZKStringSerializer implements ZkSerializer {
-
-		@Override
-		public byte[] serialize(Object data) throws ZkMarshallingError {
-			try {
-				return ((String) data).getBytes("UTF-8");
-			} catch (UnsupportedEncodingException e) {
-				throw new RuntimeException(e);
-			}
-		}
-
-		@Override
-		public Object deserialize(byte[] bytes) throws ZkMarshallingError {
-			if (bytes == null) {
-				return null;
-			} else {
-				try {
-					return new String(bytes, "UTF-8");
-				} catch (UnsupportedEncodingException e) {
-					throw new RuntimeException(e);
-				}
-			}
-		}
-	}
-
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/IncludedFetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/IncludedFetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/IncludedFetcher.java
deleted file mode 100644
index f1b9e60..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/IncludedFetcher.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.internals;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.kafka_backport.clients.consumer.CommitType;
-import org.apache.flink.kafka_backport.clients.consumer.ConsumerRecord;
-import org.apache.flink.kafka_backport.clients.consumer.ConsumerRecords;
-import org.apache.flink.kafka_backport.clients.consumer.KafkaConsumer;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.serialization.ByteArrayDeserializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-public class IncludedFetcher implements Fetcher {
-	public static Logger LOG = LoggerFactory.getLogger(IncludedFetcher.class);
-
-	public final static String POLL_TIMEOUT = "flink.kafka.consumer.poll.timeout";
-	public final static long DEFAULT_POLL_TIMEOUT = 50;
-
-	final KafkaConsumer<byte[], byte[]> fetcher;
-	final Properties props;
-	boolean running = true;
-
-	public IncludedFetcher(Properties props) {
-		this.props = props;
-		fetcher = new KafkaConsumer<byte[], byte[]>(props, null, new ByteArrayDeserializer(), new ByteArrayDeserializer());
-	}
-
-	@Override
-	public void partitionsToRead(List<TopicPartition> partitions) {
-		fetcher.subscribe(partitions.toArray(new TopicPartition[partitions.size()]));
-	}
-
-	@Override
-	public void seek(TopicPartition topicPartition, long offsetToRead) {
-		fetcher.seek(topicPartition, offsetToRead);
-	}
-
-	@Override
-	public void close() {
-		synchronized (fetcher) {
-			fetcher.close();
-		}
-	}
-
-	@Override
-	public <T> void run(SourceFunction.SourceContext<T> sourceContext, DeserializationSchema<T> valueDeserializer, long[] lastOffsets) {
-		long pollTimeout = DEFAULT_POLL_TIMEOUT;
-		if(props.contains(POLL_TIMEOUT)) {
-			pollTimeout = Long.valueOf(props.getProperty(POLL_TIMEOUT));
-		}
-		while(running) {
-			// poll is always returning a new object.
-			ConsumerRecords<byte[], byte[]> consumed;
-			synchronized (fetcher) {
-				consumed = fetcher.poll(pollTimeout);
-			}
-			if(!consumed.isEmpty()) {
-				for(ConsumerRecord<byte[], byte[]> record : consumed) {
-					// synchronize inside the loop to allow checkpoints in between
-					synchronized (sourceContext.getCheckpointLock()) {
-						T value = valueDeserializer.deserialize(record.value());
-						sourceContext.collect(value);
-						lastOffsets[record.partition()] = record.offset();
-					}
-				}
-			}
-		}
-
-		sourceContext.close();
-	}
-
-	@Override
-	public void stop() {
-		running = false;
-	}
-
-	@Override
-	public void commit(Map<TopicPartition, Long> offsetsToCommit) {
-		synchronized (fetcher) {
-			fetcher.commit(offsetsToCommit, CommitType.SYNC);
-		}
-	}
-
-}


[36/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.java
deleted file mode 100644
index c412136..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.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.internals;
-
-import kafka.admin.AdminUtils;
-
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.flink.streaming.connectors.KafkaTestBase;
-
-import org.junit.Test;
-
-import java.util.Properties;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-public class ZookeeperOffsetHandlerTest extends KafkaTestBase {
-	
-	@Test
-	public void runOffsetManipulationinZooKeeperTest() {
-		try {
-			final String topicName = "ZookeeperOffsetHandlerTest-Topic";
-			final String groupId = "ZookeeperOffsetHandlerTest-Group";
-			
-			final long offset = (long) (Math.random() * Long.MAX_VALUE);
-
-			ZkClient zkClient = createZookeeperClient();
-			AdminUtils.createTopic(zkClient, topicName, 3, 2, new Properties());
-				
-			ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, groupId, topicName, 0, offset);
-	
-			long fetchedOffset = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, groupId, topicName, 0);
-
-			zkClient.close();
-			
-			assertEquals(offset, fetchedOffset);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

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

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

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

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

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


[51/51] [abbrv] flink git commit: [FLINK-2569] [core] Add CsvReader support for Value types

Posted by se...@apache.org.
[FLINK-2569] [core] Add CsvReader support for Value types

  - Extend ValueTypeInfo to check whether the type is basic value or not
  - Extend TupleTypeInfo to support Value types
  - Add three unit tests and an integration test
  - Rename CsvReaderWithPOJOITCase to CsvReaderITCase
  - Refactor CsvReaderITCase to use collect method

This closes #1053


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

Branch: refs/heads/master
Commit: 541a06cfca90c0f47717a4c0254b73dd0eb3267f
Parents: 08500f2
Author: Chiwan Park <ch...@apache.org>
Authored: Tue Aug 25 17:53:43 2015 +0900
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Aug 27 12:40:39 2015 +0200

----------------------------------------------------------------------
 .../org/apache/flink/api/java/io/CsvReader.java |  50 +++----
 .../flink/api/java/tuple/TupleGenerator.java    |   2 +-
 .../flink/api/java/typeutils/TupleTypeInfo.java |  37 ++++-
 .../flink/api/java/typeutils/ValueTypeInfo.java |  16 +++
 .../apache/flink/api/java/io/CSVReaderTest.java |  63 +++++++-
 .../apache/flink/test/io/CsvReaderITCase.java   | 142 ++++++++++++++++++
 .../flink/test/io/CsvReaderWithPOJOITCase.java  | 144 -------------------
 7 files changed, 280 insertions(+), 174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/541a06cf/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java
index c5d6cb5..052f960 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java
@@ -373,7 +373,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0> DataSource<Tuple1<T0>> types(Class<T0> type0) {
-		TupleTypeInfo<Tuple1<T0>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0);
+		TupleTypeInfo<Tuple1<T0>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0);
 		CsvInputFormat<Tuple1<T0>> inputFormat = new CsvInputFormat<Tuple1<T0>>(path, types);
 		configureInputFormat(inputFormat, type0);
 		return new DataSource<Tuple1<T0>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -390,7 +390,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1> DataSource<Tuple2<T0, T1>> types(Class<T0> type0, Class<T1> type1) {
-		TupleTypeInfo<Tuple2<T0, T1>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1);
+		TupleTypeInfo<Tuple2<T0, T1>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1);
 		CsvInputFormat<Tuple2<T0, T1>> inputFormat = new CsvInputFormat<Tuple2<T0, T1>>(path, types);
 		configureInputFormat(inputFormat, type0, type1);
 		return new DataSource<Tuple2<T0, T1>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -408,7 +408,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2> DataSource<Tuple3<T0, T1, T2>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2) {
-		TupleTypeInfo<Tuple3<T0, T1, T2>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2);
+		TupleTypeInfo<Tuple3<T0, T1, T2>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2);
 		CsvInputFormat<Tuple3<T0, T1, T2>> inputFormat = new CsvInputFormat<Tuple3<T0, T1, T2>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2);
 		return new DataSource<Tuple3<T0, T1, T2>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -427,7 +427,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3> DataSource<Tuple4<T0, T1, T2, T3>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3) {
-		TupleTypeInfo<Tuple4<T0, T1, T2, T3>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3);
+		TupleTypeInfo<Tuple4<T0, T1, T2, T3>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3);
 		CsvInputFormat<Tuple4<T0, T1, T2, T3>> inputFormat = new CsvInputFormat<Tuple4<T0, T1, T2, T3>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3);
 		return new DataSource<Tuple4<T0, T1, T2, T3>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -447,7 +447,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4> DataSource<Tuple5<T0, T1, T2, T3, T4>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4) {
-		TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4);
+		TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4);
 		CsvInputFormat<Tuple5<T0, T1, T2, T3, T4>> inputFormat = new CsvInputFormat<Tuple5<T0, T1, T2, T3, T4>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4);
 		return new DataSource<Tuple5<T0, T1, T2, T3, T4>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -468,7 +468,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5> DataSource<Tuple6<T0, T1, T2, T3, T4, T5>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5) {
-		TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5);
+		TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5);
 		CsvInputFormat<Tuple6<T0, T1, T2, T3, T4, T5>> inputFormat = new CsvInputFormat<Tuple6<T0, T1, T2, T3, T4, T5>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5);
 		return new DataSource<Tuple6<T0, T1, T2, T3, T4, T5>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -490,7 +490,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6> DataSource<Tuple7<T0, T1, T2, T3, T4, T5, T6>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6) {
-		TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6);
+		TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6);
 		CsvInputFormat<Tuple7<T0, T1, T2, T3, T4, T5, T6>> inputFormat = new CsvInputFormat<Tuple7<T0, T1, T2, T3, T4, T5, T6>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6);
 		return new DataSource<Tuple7<T0, T1, T2, T3, T4, T5, T6>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -513,7 +513,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7> DataSource<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7) {
-		TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7);
+		TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7);
 		CsvInputFormat<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> inputFormat = new CsvInputFormat<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7);
 		return new DataSource<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -537,7 +537,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8> DataSource<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8) {
-		TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8);
+		TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8);
 		CsvInputFormat<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> inputFormat = new CsvInputFormat<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8);
 		return new DataSource<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -562,7 +562,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> DataSource<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9) {
-		TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9);
+		TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9);
 		CsvInputFormat<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> inputFormat = new CsvInputFormat<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9);
 		return new DataSource<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -588,7 +588,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> DataSource<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10) {
-		TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10);
+		TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10);
 		CsvInputFormat<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> inputFormat = new CsvInputFormat<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10);
 		return new DataSource<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -615,7 +615,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11> DataSource<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11) {
-		TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11);
+		TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11);
 		CsvInputFormat<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> inputFormat = new CsvInputFormat<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11);
 		return new DataSource<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -643,7 +643,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12> DataSource<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12) {
-		TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12);
+		TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12);
 		CsvInputFormat<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> inputFormat = new CsvInputFormat<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12);
 		return new DataSource<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -672,7 +672,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13> DataSource<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13) {
-		TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13);
+		TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13);
 		CsvInputFormat<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> inputFormat = new CsvInputFormat<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13);
 		return new DataSource<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -702,7 +702,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14> DataSource<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14) {
-		TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14);
+		TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14);
 		CsvInputFormat<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> inputFormat = new CsvInputFormat<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14);
 		return new DataSource<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -733,7 +733,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15> DataSource<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15) {
-		TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15);
+		TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15);
 		CsvInputFormat<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> inputFormat = new CsvInputFormat<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15);
 		return new DataSource<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -765,7 +765,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16> DataSource<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15, Class<T16> type16) {
-		TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16);
+		TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16);
 		CsvInputFormat<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> inputFormat = new CsvInputFormat<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16);
 		return new DataSource<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -798,7 +798,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17> DataSource<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17) {
-		TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17);
+		TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17);
 		CsvInputFormat<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> inputFormat = new CsvInputFormat<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17);
 		return new DataSource<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -832,7 +832,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18> DataSource<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17, Class<T18> type18) {
-		TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18);
+		TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18);
 		CsvInputFormat<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> inputFormat = new CsvInputFormat<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18);
 		return new DataSource<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -867,7 +867,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19> DataSource<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17, Class<T18> type18, Class<T19> type19) {
-		TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19);
+		TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19);
 		CsvInputFormat<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> inputFormat = new CsvInputFormat<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19);
 		return new DataSource<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -903,7 +903,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20> DataSource<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17, Class<T18> type18, Class<T19> type19, Class<T20> type20) {
-		TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20);
+		TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20);
 		CsvInputFormat<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> inputFormat = new CsvInputFormat<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20);
 		return new DataSource<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -940,7 +940,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21> DataSource<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17, Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21) {
-		TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21);
+		TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21);
 		CsvInputFormat<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> inputFormat = new CsvInputFormat<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21);
 		return new DataSource<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -978,7 +978,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22> DataSource<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17, Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21, Class<T22> type22) {
-		TupleTypeInfo<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21, type22);
+		TupleTypeInfo<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21, type22);
 		CsvInputFormat<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>> inputFormat = new CsvInputFormat<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21, type22);
 		return new DataSource<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -1017,7 +1017,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23> DataSource<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17, Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21, Class<T22> type22, Class<T23> type23) {
-		TupleTypeInfo<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21, type22, type23);
+		TupleTypeInfo<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21, type22, type23);
 		CsvInputFormat<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>> inputFormat = new CsvInputFormat<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21, type22, type23);
 		return new DataSource<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>>(executionContext, inputFormat, types, Utils.getCallLocationName());
@@ -1057,7 +1057,7 @@ public class CsvReader {
 	 * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.
 	 */
 	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24> DataSource<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>> types(Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4, Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9, Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13, Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17, Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21, Class<T22> type22, Class<T23> type23, Class<T24> type24) {
-		TupleTypeInfo<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>> types = TupleTypeInfo.getBasicTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21, type22, type23, type24);
+		TupleTypeInfo<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21, type22, type23, type24);
 		CsvInputFormat<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>> inputFormat = new CsvInputFormat<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>>(path, types);
 		configureInputFormat(inputFormat, type0, type1, type2, type3, type4, type5, type6, type7, type8, type9, type10, type11, type12, type13, type14, type15, type16, type17, type18, type19, type20, type21, type22, type23, type24);
 		return new DataSource<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>>(executionContext, inputFormat, types, Utils.getCallLocationName());

http://git-wip-us.apache.org/repos/asf/flink/blob/541a06cf/flink-java/src/main/java/org/apache/flink/api/java/tuple/TupleGenerator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/tuple/TupleGenerator.java b/flink-java/src/main/java/org/apache/flink/api/java/tuple/TupleGenerator.java
index 6ab02e4..f306fe0 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/tuple/TupleGenerator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/tuple/TupleGenerator.java
@@ -457,7 +457,7 @@ class TupleGenerator {
 			// get TupleTypeInfo
 			sb.append("\t\tTupleTypeInfo<Tuple" + numFields + "<");
 			appendTupleTypeGenerics(sb, numFields);
-			sb.append(">> types = TupleTypeInfo.getBasicTupleTypeInfo(");
+			sb.append(">> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo(");
 			for (int i = 0; i < numFields; i++) {
 				if (i > 0) {
 					sb.append(", ");

http://git-wip-us.apache.org/repos/asf/flink/blob/541a06cf/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
index a671bd4..618b190 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
@@ -21,6 +21,7 @@ package org.apache.flink.api.java.typeutils;
 import java.util.Arrays;
 
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 //CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator
@@ -31,6 +32,7 @@ import org.apache.flink.api.java.typeutils.runtime.Tuple0Serializer;
 //CHECKSTYLE.ON: AvoidStarImport
 import org.apache.flink.api.java.typeutils.runtime.TupleComparator;
 import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.types.Value;
 
 /**
  * A {@link TypeInformation} for the tuple types of the Java API.
@@ -159,7 +161,7 @@ public final class TupleTypeInfo<T extends Tuple> extends TupleTypeInfoBase<T> {
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
+
 	public static <X extends Tuple> TupleTypeInfo<X> getBasicTupleTypeInfo(Class<?>... basicTypes) {
 		if (basicTypes == null || basicTypes.length == 0) {
 			throw new IllegalArgumentException();
@@ -178,9 +180,40 @@ public final class TupleTypeInfo<T extends Tuple> extends TupleTypeInfoBase<T> {
 			}
 			infos[i] = info;
 		}
-		
+
 		@SuppressWarnings("unchecked")
 		TupleTypeInfo<X> tupleInfo = (TupleTypeInfo<X>) new TupleTypeInfo<Tuple>(infos);
 		return tupleInfo;
 	}
+
+	@SuppressWarnings("unchecked")
+	public static <X extends Tuple> TupleTypeInfo<X> getBasicAndBasicValueTupleTypeInfo(Class<?>... basicTypes) {
+		if (basicTypes == null || basicTypes.length == 0) {
+			throw new IllegalArgumentException();
+		}
+
+		TypeInformation<?>[] infos = new TypeInformation<?>[basicTypes.length];
+		for (int i = 0; i < infos.length; i++) {
+			Class<?> type = basicTypes[i];
+			if (type == null) {
+				throw new IllegalArgumentException("Type at position " + i + " is null.");
+			}
+
+			TypeInformation<?> info = BasicTypeInfo.getInfoFor(type);
+			if (info == null) {
+				try {
+					info = ValueTypeInfo.getValueTypeInfo((Class<Value>) type);
+					if (!((ValueTypeInfo<?>) info).isBasicValueType()) {
+						throw new IllegalArgumentException("Type at position " + i + " is not a basic or value type.");
+					}
+				} catch (ClassCastException | InvalidTypesException e) {
+					throw new IllegalArgumentException("Type at position " + i + " is not a basic or value type.", e);
+				}
+			}
+			infos[i] = info;
+		}
+
+
+		return (TupleTypeInfo<X>) new TupleTypeInfo<>(infos);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/541a06cf/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
index 8288b45..e61acd8 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
@@ -28,7 +28,17 @@ import org.apache.flink.api.java.typeutils.runtime.CopyableValueComparator;
 import org.apache.flink.api.java.typeutils.runtime.CopyableValueSerializer;
 import org.apache.flink.api.java.typeutils.runtime.ValueComparator;
 import org.apache.flink.api.java.typeutils.runtime.ValueSerializer;
+import org.apache.flink.types.BooleanValue;
+import org.apache.flink.types.ByteValue;
+import org.apache.flink.types.CharValue;
 import org.apache.flink.types.CopyableValue;
+import org.apache.flink.types.DoubleValue;
+import org.apache.flink.types.FloatValue;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.types.ShortValue;
+import org.apache.flink.types.StringValue;
 import org.apache.flink.types.Value;
 
 /**
@@ -75,6 +85,12 @@ public class ValueTypeInfo<T extends Value> extends TypeInformation<T> implement
 		return false;
 	}
 
+	public boolean isBasicValueType() {
+		return type.equals(StringValue.class) || type.equals(ByteValue.class) || type.equals(ShortValue.class) || type.equals(CharValue.class) ||
+				type.equals(DoubleValue.class) || type.equals(FloatValue.class) || type.equals(IntValue.class) || type.equals(LongValue.class) ||
+				type.equals(NullValue.class) || type.equals(BooleanValue.class);
+	}
+
 	@Override
 	public boolean isTupleType() {
 		return false;

http://git-wip-us.apache.org/repos/asf/flink/blob/541a06cf/flink-java/src/test/java/org/apache/flink/api/java/io/CSVReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/CSVReaderTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/CSVReaderTest.java
index 6676cd1..8b12315 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/io/CSVReaderTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/io/CSVReaderTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.api.java.io;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
+import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
@@ -28,10 +29,21 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.operators.DataSource;
 import org.apache.flink.api.java.tuple.Tuple4;
 import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple8;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.api.java.typeutils.ValueTypeInfo;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.types.BooleanValue;
+import org.apache.flink.types.ByteValue;
+import org.apache.flink.types.CharValue;
+import org.apache.flink.types.DoubleValue;
+import org.apache.flink.types.FloatValue;
+import org.apache.flink.types.IntValue;
 import org.apache.flink.types.LongValue;
+import org.apache.flink.types.ShortValue;
 import org.apache.flink.types.StringValue;
+import org.apache.flink.types.Value;
 import org.junit.Assert;
 import org.junit.Test;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -190,7 +202,7 @@ public class CSVReaderTest {
 		}
 		
 		CsvInputFormat<?> inputFormat = (CsvInputFormat<?>) items.getInputFormat();
-		Assert.assertArrayEquals(new Class<?>[] {Integer.class, String.class, Double.class, String.class}, inputFormat.getFieldTypes());
+		Assert.assertArrayEquals(new Class<?>[]{Integer.class, String.class, Double.class, String.class}, inputFormat.getFieldTypes());
 	}
 	
 	@Test
@@ -211,7 +223,7 @@ public class CSVReaderTest {
 		Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, tinfo.getTypeAt(3));
 		
 		CsvInputFormat<?> inputFormat = (CsvInputFormat<?>) sitems.getInputFormat();
-		Assert.assertArrayEquals(new Class<?>[] {Integer.class, String.class, Double.class, String.class}, inputFormat.getFieldTypes());
+		Assert.assertArrayEquals(new Class<?>[]{Integer.class, String.class, Double.class, String.class}, inputFormat.getFieldTypes());
 	}
 	
 	@Test
@@ -250,6 +262,31 @@ public class CSVReaderTest {
 			// okay.
 		}
 	}
+
+	@Test
+	public void testWithValueType() throws Exception {
+		CsvReader reader = getCsvReader();
+		DataSource<Tuple8<StringValue, BooleanValue, ByteValue, ShortValue, IntValue, LongValue, FloatValue, DoubleValue>> items =
+				reader.types(StringValue.class, BooleanValue.class, ByteValue.class, ShortValue.class, IntValue.class, LongValue.class, FloatValue.class, DoubleValue.class);
+		TypeInformation<?> info = items.getType();
+
+		Assert.assertEquals(true, info.isTupleType());
+		Assert.assertEquals(Tuple8.class, info.getTypeClass());
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testWithInvalidValueType1() throws Exception {
+		CsvReader reader = getCsvReader();
+		// CsvReader doesn't support CharValue
+		reader.types(CharValue.class);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testWithInvalidValueType2() throws Exception {
+		CsvReader reader = getCsvReader();
+		// CsvReader doesn't support custom Value type
+		reader.types(ValueItem.class);
+	}
 	
 	private static CsvReader getCsvReader() {
 		return new CsvReader("/some/none/existing/path", ExecutionEnvironment.createLocalEnvironment(1));
@@ -274,4 +311,26 @@ public class CSVReaderTest {
 	public static class FinalItem extends PartialItem<String, StringValue, LongValue> {
 		private static final long serialVersionUID = 1L;
 	}
+
+	public static class ValueItem implements Value {
+		private int v1;
+
+		public int getV1() {
+			return v1;
+		}
+
+		public void setV1(int v1) {
+			this.v1 = v1;
+		}
+
+		@Override
+		public void write(DataOutputView out) throws IOException {
+			out.writeInt(v1);
+		}
+
+		@Override
+		public void read(DataInputView in) throws IOException {
+			v1 = in.readInt();
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/541a06cf/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
new file mode 100644
index 0000000..4d20b54
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.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.test.io;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple8;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.BooleanValue;
+import org.apache.flink.types.ByteValue;
+import org.apache.flink.types.DoubleValue;
+import org.apache.flink.types.FloatValue;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.ShortValue;
+import org.apache.flink.types.StringValue;
+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.List;
+
+@RunWith(Parameterized.class)
+public class CsvReaderITCase extends MultipleProgramsTestBase {
+	private String expected;
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	public CsvReaderITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	private String createInputData(String data) throws Exception {
+		File file = tempFolder.newFile("input");
+		Files.write(data, file, Charsets.UTF_8);
+
+		return file.toURI().toString();
+	}
+
+	@Test
+	public void testPOJOType() throws Exception {
+		final String inputData = "ABC,2.20,3\nDEF,5.1,5\nDEF,3.30,1\nGHI,3.30,10";
+		final String dataPath = createInputData(inputData);
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJOItem> data = env.readCsvFile(dataPath).pojoType(POJOItem.class, new String[]{"f1", "f3", "f2"});
+		List<POJOItem> result = data.collect();
+
+		expected = "ABC,3,2.20\nDEF,5,5.10\nDEF,1,3.30\nGHI,10,3.30";
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testPOJOTypeWithFieldsOrder() throws Exception {
+		final String inputData = "2.20,ABC,3\n5.1,DEF,5\n3.30,DEF,1\n3.30,GHI,10";
+		final String dataPath = createInputData(inputData);
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJOItem> data = env.readCsvFile(dataPath).pojoType(POJOItem.class, new String[]{"f3", "f1", "f2"});
+		List<POJOItem> result = data.collect();
+
+		expected = "ABC,3,2.20\nDEF,5,5.10\nDEF,1,3.30\nGHI,10,3.30";
+		compareResultAsText(result, expected);
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void testPOJOTypeWithoutFieldsOrder() throws Exception {
+		final String inputData = "";
+		final String dataPath = createInputData(inputData);
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		env.readCsvFile(dataPath).pojoType(POJOItem.class, null);
+	}
+
+	@Test
+	public void testPOJOTypeWithFieldsOrderAndFieldsSelection() throws Exception {
+		final String inputData = "3,2.20,ABC\n5,5.1,DEF\n1,3.30,DEF\n10,3.30,GHI";
+		final String dataPath = createInputData(inputData);
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJOItem> data = env.readCsvFile(dataPath).includeFields(true, false, true).pojoType(POJOItem.class, new String[]{"f2", "f1"});
+		List<POJOItem> result = data.collect();
+
+		expected = "ABC,3,0.00\nDEF,5,0.00\nDEF,1,0.00\nGHI,10,0.00";
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testValueTypes() throws Exception {
+		final String inputData = "ABC,true,1,2,3,4,5.0,6.0\nBCD,false,1,2,3,4,5.0,6.0";
+		final String dataPath = createInputData(inputData);
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple8<StringValue, BooleanValue, ByteValue, ShortValue, IntValue, LongValue, FloatValue, DoubleValue>> data =
+				env.readCsvFile(dataPath).types(StringValue.class, BooleanValue.class, ByteValue.class, ShortValue.class, IntValue.class, LongValue.class, FloatValue.class, DoubleValue.class);
+		List<Tuple8<StringValue, BooleanValue, ByteValue, ShortValue, IntValue, LongValue, FloatValue, DoubleValue>> result = data.collect();
+
+		expected = inputData;
+		compareResultAsTuples(result, expected);
+	}
+
+	public static class POJOItem {
+		public String f1;
+		private int f2;
+		public double f3;
+
+		public int getF2() {
+			return f2;
+		}
+
+		public void setF2(int f2) {
+			this.f2 = f2;
+		}
+
+		@Override
+		public String toString() {
+			return String.format("%s,%d,%.02f", f1, f2, f3);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/541a06cf/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderWithPOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderWithPOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderWithPOJOITCase.java
deleted file mode 100644
index 6a614e9..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderWithPOJOITCase.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.io;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.After;
-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 static org.junit.Assert.fail;
-
-@RunWith(Parameterized.class)
-public class CsvReaderWithPOJOITCase extends MultipleProgramsTestBase {
-	private String resultPath;
-	private String expected;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	public CsvReaderWithPOJOITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Before
-	public void before() throws Exception {
-		resultPath = tempFolder.newFile("result").toURI().toString();
-	}
-
-	@After
-	public void after() throws Exception {
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-
-	private String createInputData(String data) throws Exception {
-		File file = tempFolder.newFile("input");
-		Files.write(data, file, Charsets.UTF_8);
-
-		return file.toURI().toString();
-	}
-
-	@Test
-	public void testPOJOType() throws Exception {
-		final String inputData = "ABC,2.20,3\nDEF,5.1,5\nDEF,3.30,1\nGHI,3.30,10";
-		final String dataPath = createInputData(inputData);
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJOItem> data = env.readCsvFile(dataPath).pojoType(POJOItem.class, new String[]{"f1", "f3", "f2"});
-		data.writeAsText(resultPath);
-
-		env.execute();
-
-		expected = "ABC,3,2.20\nDEF,5,5.10\nDEF,1,3.30\nGHI,10,3.30";
-	}
-
-	@Test
-	public void testPOJOTypeWithFieldsOrder() throws Exception {
-		final String inputData = "2.20,ABC,3\n5.1,DEF,5\n3.30,DEF,1\n3.30,GHI,10";
-		final String dataPath = createInputData(inputData);
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJOItem> data = env.readCsvFile(dataPath).pojoType(POJOItem.class, new String[]{"f3", "f1", "f2"});
-		data.writeAsText(resultPath);
-
-		env.execute();
-
-		expected = "ABC,3,2.20\nDEF,5,5.10\nDEF,1,3.30\nGHI,10,3.30";
-	}
-
-	@Test
-	public void testPOJOTypeWithoutFieldsOrder() throws Exception {
-		final String inputData = "";
-		final String dataPath = createInputData(inputData);
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		try {
-			env.readCsvFile(dataPath).pojoType(POJOItem.class, null);
-			fail("POJO type without fields order must raise NullPointerException!");
-		} catch (NullPointerException e) {
-			// success
-		}
-
-		expected = "";
-		resultPath = dataPath;
-	}
-
-	@Test
-	public void testPOJOTypeWithFieldsOrderAndFieldsSelection() throws Exception {
-		final String inputData = "3,2.20,ABC\n5,5.1,DEF\n1,3.30,DEF\n10,3.30,GHI";
-		final String dataPath = createInputData(inputData);
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJOItem> data = env.readCsvFile(dataPath).includeFields(true, false, true).pojoType(POJOItem.class, new String[]{"f2", "f1"});
-		data.writeAsText(resultPath);
-
-		env.execute();
-
-		expected = "ABC,3,0.00\nDEF,5,0.00\nDEF,1,0.00\nGHI,10,0.00";
-	}
-
-	public static class POJOItem {
-		public String f1;
-		private int f2;
-		public double f3;
-
-		public int getF2() {
-			return f2;
-		}
-
-		public void setF2(int f2) {
-			this.f2 = f2;
-		}
-
-		@Override
-		public String toString() {
-			return String.format("%s,%d,%.02f", f1, f2, f3);
-		}
-	}
-}


[37/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer082.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer082.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer082.java
deleted file mode 100644
index 1540f8f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer082.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;
-
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Creates a Kafka consumer compatible with reading from Kafka 0.8.2.x brokers.
- * The consumer will use the new Kafka consumer API (early Flink backport version),
- * and manually commit offsets partition offsets to ZooKeeper.
- *
- * @param <T> The type of elements produced by this consumer.
- */
-public class FlinkKafkaConsumer082<T> extends FlinkKafkaConsumer<T> {
-
-	private static final long serialVersionUID = -8450689820627198228L;
-
-	/**
-	 * Creates a new Kafka 0.8.2.x streaming source consumer.
-	 * 
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param valueDeserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects. 
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer082(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		super(topic, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.NEW_HIGH_LEVEL);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer083.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer083.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer083.java
deleted file mode 100644
index 6ed2930..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/FlinkKafkaConsumer083.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;
-
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Creates a Kafka consumer compatible with reading from Kafka 0.8.3.x brokers.
- * The consumer will use the new Kafka consumer API (early Flink backport version),
- * and lets Kafka handle the offset committing internally.
- * 
- * @param <T> The type of elements produced by this consumer.
- */
-public class FlinkKafkaConsumer083<T> extends FlinkKafkaConsumer<T> {
-
-	private static final long serialVersionUID = 1126432820518992927L;
-
-	/**
-	 * Creates a new Kafka 0.8.3.x streaming source consumer.
-	 *
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param valueDeserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects. 
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer083(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		super(topic, valueDeserializer, props, OffsetStore.KAFKA, FetcherType.NEW_HIGH_LEVEL);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/KafkaSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/KafkaSink.java
deleted file mode 100644
index 51129ed..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/KafkaSink.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors;
-
-import com.google.common.base.Preconditions;
-import kafka.javaapi.producer.Producer;
-import kafka.producer.KeyedMessage;
-import kafka.producer.ProducerConfig;
-import kafka.serializer.DefaultEncoder;
-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.util.serialization.SerializationSchema;
-import org.apache.flink.util.NetUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-import java.util.Properties;
-
-
-/**
- * Sink that emits its inputs to a Kafka topic.
- *
- * @param <IN>
- * 		Type of the sink input
- */
-public class KafkaSink<IN> extends RichSinkFunction<IN> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(KafkaSink.class);
-
-	private Producer<IN, byte[]> producer;
-	private Properties userDefinedProperties;
-	private String topicId;
-	private String brokerList;
-	private SerializationSchema<IN, byte[]> schema;
-	private SerializableKafkaPartitioner partitioner;
-	private Class<? extends SerializableKafkaPartitioner> partitionerClass = null;
-
-	/**
-	 * Creates a KafkaSink for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 *			Addresses of the brokers
-	 * @param topicId
-	 * 		ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 		User defined serialization schema.
-	 */
-	public KafkaSink(String brokerList, String topicId, SerializationSchema<IN, byte[]> serializationSchema) {
-		this(brokerList, topicId, new Properties(), serializationSchema);
-	}
-
-	/**
-	 * Creates a KafkaSink for a given topic with custom Producer configuration.
-	 * If you use this constructor, the broker should be set with the "metadata.broker.list"
-	 * configuration.
-	 *
-	 * @param brokerList
-	 * 		Addresses of the brokers
-	 * @param topicId
-	 * 		ID of the Kafka topic.
-	 * @param producerConfig
-	 * 		Configurations of the Kafka producer
-	 * @param serializationSchema
-	 * 		User defined serialization schema.
-	 */
-	public KafkaSink(String brokerList, String topicId, Properties producerConfig, SerializationSchema<IN, byte[]> serializationSchema) {
-		String[] elements = brokerList.split(",");
-		for(String broker: elements) {
-			NetUtils.ensureCorrectHostnamePort(broker);
-		}
-		Preconditions.checkNotNull(topicId, "TopicID not set");
-
-		this.brokerList = brokerList;
-		this.topicId = topicId;
-		this.schema = serializationSchema;
-		this.partitionerClass = null;
-		this.userDefinedProperties = producerConfig;
-	}
-
-	/**
-	 * Creates a KafkaSink for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 * @param topicId
-	 * 		ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 		User defined serialization schema.
-	 * @param partitioner
-	 * 		User defined partitioner.
-	 */
-	public KafkaSink(String brokerList, String topicId, SerializationSchema<IN, byte[]> serializationSchema, SerializableKafkaPartitioner partitioner) {
-		this(brokerList, topicId, serializationSchema);
-		ClosureCleaner.ensureSerializable(partitioner);
-		this.partitioner = partitioner;
-	}
-
-	public KafkaSink(String brokerList,
-					String topicId,
-					SerializationSchema<IN, byte[]> serializationSchema,
-					Class<? extends SerializableKafkaPartitioner> partitioner) {
-		this(brokerList, topicId, serializationSchema);
-		this.partitionerClass = partitioner;
-	}
-
-	/**
-	 * Initializes the connection to Kafka.
-	 */
-	@Override
-	public void open(Configuration configuration) {
-
-		Properties properties = new Properties();
-
-		properties.put("metadata.broker.list", brokerList);
-		properties.put("request.required.acks", "-1");
-		properties.put("message.send.max.retries", "10");
-
-		properties.put("serializer.class", DefaultEncoder.class.getCanonicalName());
-
-		// this will not be used as the key will not be serialized
-		properties.put("key.serializer.class", DefaultEncoder.class.getCanonicalName());
-
-		for (Map.Entry<Object, Object> propertiesEntry : userDefinedProperties.entrySet()) {
-			properties.put(propertiesEntry.getKey(), propertiesEntry.getValue());
-		}
-
-		if (partitioner != null) {
-			properties.put("partitioner.class", PartitionerWrapper.class.getCanonicalName());
-			// java serialization will do the rest.
-			properties.put(PartitionerWrapper.SERIALIZED_WRAPPER_NAME, partitioner);
-		}
-		if (partitionerClass != null) {
-			properties.put("partitioner.class", partitionerClass);
-		}
-
-		ProducerConfig config = new ProducerConfig(properties);
-
-		try {
-			producer = new Producer<IN, byte[]>(config);
-		} catch (NullPointerException e) {
-			throw new RuntimeException("Cannot connect to Kafka broker " + brokerList, e);
-		}
-	}
-
-	/**
-	 * Called when new data arrives to the sink, and forwards it to Kafka.
-	 *
-	 * @param next
-	 * 		The incoming data
-	 */
-	@Override
-	public void invoke(IN next) {
-		byte[] serialized = schema.serialize(next);
-
-		// Sending message without serializable key.
-		producer.send(new KeyedMessage<IN, byte[]>(topicId, null, next, serialized));
-	}
-
-	@Override
-	public void close() {
-		if (producer != null) {
-			producer.close();
-		}
-	}
-
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/SerializableKafkaPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/SerializableKafkaPartitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/SerializableKafkaPartitioner.java
deleted file mode 100644
index aff0c3a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/SerializableKafkaPartitioner.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;
-
-import kafka.producer.Partitioner;
-
-import java.io.Serializable;
-
-public interface SerializableKafkaPartitioner extends Serializable, Partitioner {
-
-}

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka082ITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka082ITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka082ITCase.java
deleted file mode 100644
index 2f80fcb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/Kafka082ITCase.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;
-
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import org.junit.Ignore;
-import org.junit.Test;
-
-import java.util.Properties;
-
-
-public class Kafka082ITCase extends KafkaConsumerTestBase {
-	
-	@Override
-	protected <T> FlinkKafkaConsumer<T> getConsumer(String topic, DeserializationSchema<T> deserializationSchema, Properties props) {
-		return new FlinkKafkaConsumer082<T>(topic, deserializationSchema, props);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Suite of Tests
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testCheckpointing() {
-		runCheckpointingTest();
-	}
-
-	@Test
-	public void testOffsetInZookeeper() {
-		runOffsetInZookeeperValidationTest();
-	}
-
-	@Test
-	public void testConcurrentProducerConsumerTopology() {
-		runSimpleConcurrentProducerConsumerTopology();
-	}
-
-	// --- canceling / failures ---
-
-	@Test
-	public void testCancelingEmptyTopic() {
-		runCancelingOnEmptyInputTest();
-	}
-
-	@Test
-	public void testCancelingFullTopic() {
-		runCancelingOnFullInputTest();
-	}
-
-	@Test
-	public void testFailOnDeploy() {
-		runFailOnDeployTest();
-	}
-
-	// --- source to partition mappings and exactly once ---
-
-	@Test
-	public void testOneToOneSources() {
-		runOneToOneExactlyOnceTest();
-	}
-
-	@Test
-	public void testOneSourceMultiplePartitions() {
-		runOneSourceMultiplePartitionsExactlyOnceTest();
-	}
-
-	@Test
-	public void testMultipleSourcesOnePartition() {
-		runMultipleSourcesOnePartitionExactlyOnceTest();
-	}
-
-	// --- broker failure ---
-
-	@Test
-	public void testBrokerFailure() {
-		runBrokerFailureTest();
-	}
-
-	// --- special executions ---
-
-	@Test
-	@Ignore("this does not work with the new consumer")
-	public void testBigRecordJob() {
-		runBigRecordTestTopology();
-	}
-}

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

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


[47/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Coordinator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Coordinator.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Coordinator.java
deleted file mode 100644
index 1746c22..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Coordinator.java
+++ /dev/null
@@ -1,791 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer.internals;
-
-import org.apache.flink.kafka_backport.clients.ClientResponse;
-import org.apache.flink.kafka_backport.clients.consumer.CommitType;
-import org.apache.flink.kafka_backport.clients.consumer.ConsumerCommitCallback;
-import org.apache.flink.kafka_backport.common.KafkaException;
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.apache.flink.kafka_backport.common.Node;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.errors.DisconnectException;
-import org.apache.flink.kafka_backport.common.metrics.Measurable;
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-import org.apache.flink.kafka_backport.common.metrics.Metrics;
-import org.apache.flink.kafka_backport.common.metrics.Sensor;
-import org.apache.flink.kafka_backport.common.metrics.stats.Avg;
-import org.apache.flink.kafka_backport.common.metrics.stats.Count;
-import org.apache.flink.kafka_backport.common.metrics.stats.Max;
-import org.apache.flink.kafka_backport.common.metrics.stats.Rate;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-import org.apache.flink.kafka_backport.common.requests.ConsumerMetadataRequest;
-import org.apache.flink.kafka_backport.common.requests.ConsumerMetadataResponse;
-import org.apache.flink.kafka_backport.common.requests.HeartbeatRequest;
-import org.apache.flink.kafka_backport.common.requests.HeartbeatResponse;
-import org.apache.flink.kafka_backport.common.requests.JoinGroupRequest;
-import org.apache.flink.kafka_backport.common.requests.JoinGroupResponse;
-import org.apache.flink.kafka_backport.common.requests.OffsetCommitRequest;
-import org.apache.flink.kafka_backport.common.requests.OffsetCommitResponse;
-import org.apache.flink.kafka_backport.common.requests.OffsetFetchRequest;
-import org.apache.flink.kafka_backport.common.requests.OffsetFetchResponse;
-import org.apache.flink.kafka_backport.common.utils.Time;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-/**
- * This class manages the coordination process with the consumer coordinator.
- */
-public final class Coordinator {
-
-    private static final Logger log = LoggerFactory.getLogger(Coordinator.class);
-
-    private final ConsumerNetworkClient client;
-    private final Time time;
-    private final String groupId;
-    private final Heartbeat heartbeat;
-    private final HeartbeatTask heartbeatTask;
-    private final int sessionTimeoutMs;
-    private final String assignmentStrategy;
-    private final SubscriptionState subscriptions;
-    private final CoordinatorMetrics sensors;
-    private final long requestTimeoutMs;
-    private final long retryBackoffMs;
-    private final RebalanceCallback rebalanceCallback;
-    private Node consumerCoordinator;
-    private String consumerId;
-    private int generation;
-
-
-    /**
-     * Initialize the coordination manager.
-     */
-    public Coordinator(ConsumerNetworkClient client,
-                       String groupId,
-                       int sessionTimeoutMs,
-                       String assignmentStrategy,
-                       SubscriptionState subscriptions,
-                       Metrics metrics,
-                       String metricGrpPrefix,
-                       Map<String, String> metricTags,
-                       Time time,
-                       long requestTimeoutMs,
-                       long retryBackoffMs,
-                       RebalanceCallback rebalanceCallback) {
-
-        this.client = client;
-        this.time = time;
-        this.generation = -1;
-        this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-        this.groupId = groupId;
-        this.consumerCoordinator = null;
-        this.subscriptions = subscriptions;
-        this.sessionTimeoutMs = sessionTimeoutMs;
-        this.assignmentStrategy = assignmentStrategy;
-        this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds());
-        this.heartbeatTask = new HeartbeatTask();
-        this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
-        this.requestTimeoutMs = requestTimeoutMs;
-        this.retryBackoffMs = retryBackoffMs;
-        this.rebalanceCallback = rebalanceCallback;
-    }
-
-    /**
-     * Refresh the committed offsets for provided partitions.
-     */
-    public void refreshCommittedOffsetsIfNeeded() {
-        if (subscriptions.refreshCommitsNeeded()) {
-            Map<TopicPartition, Long> offsets = fetchCommittedOffsets(subscriptions.assignedPartitions());
-            for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet()) {
-                TopicPartition tp = entry.getKey();
-                this.subscriptions.committed(tp, entry.getValue());
-            }
-            this.subscriptions.commitsRefreshed();
-        }
-    }
-
-    /**
-     * Fetch the current committed offsets from the coordinator for a set of partitions.
-     * @param partitions The partitions to fetch offsets for
-     * @return A map from partition to the committed offset
-     */
-    public Map<TopicPartition, Long> fetchCommittedOffsets(Set<TopicPartition> partitions) {
-        while (true) {
-            ensureCoordinatorKnown();
-            ensurePartitionAssignment();
-
-            // contact coordinator to fetch committed offsets
-            RequestFuture<Map<TopicPartition, Long>> future = sendOffsetFetchRequest(partitions);
-            client.poll(future);
-
-            if (future.succeeded())
-                return future.value();
-
-            if (!future.isRetriable())
-                throw future.exception();
-
-            Utils.sleep(retryBackoffMs);
-        }
-    }
-
-    /**
-     * Ensure that we have a valid partition assignment from the coordinator.
-     */
-    public void ensurePartitionAssignment() {
-        if (!subscriptions.partitionAssignmentNeeded())
-            return;
-
-        // execute the user's callback before rebalance
-        log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions());
-        try {
-            Set<TopicPartition> revoked = new HashSet<TopicPartition>(subscriptions.assignedPartitions());
-            rebalanceCallback.onPartitionsRevoked(revoked);
-        } catch (Exception e) {
-            log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
-                    + " failed on partition revocation: ", e);
-        }
-
-        reassignPartitions();
-
-        // execute the user's callback after rebalance
-        log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions());
-        try {
-            Set<TopicPartition> assigned = new HashSet<TopicPartition>(subscriptions.assignedPartitions());
-            rebalanceCallback.onPartitionsAssigned(assigned);
-        } catch (Exception e) {
-            log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
-                    + " failed on partition assignment: ", e);
-        }
-    }
-
-    private void reassignPartitions() {
-        while (subscriptions.partitionAssignmentNeeded()) {
-            ensureCoordinatorKnown();
-
-            // ensure that there are no pending requests to the coordinator. This is important
-            // in particular to avoid resending a pending JoinGroup request.
-            if (client.pendingRequestCount(this.consumerCoordinator) > 0) {
-                client.awaitPendingRequests(this.consumerCoordinator);
-                continue;
-            }
-
-            RequestFuture<Void> future = sendJoinGroupRequest();
-            client.poll(future);
-
-            if (future.failed()) {
-                if (!future.isRetriable())
-                    throw future.exception();
-                Utils.sleep(retryBackoffMs);
-            }
-        }
-    }
-
-    /**
-     * Block until the coordinator for this group is known.
-     */
-    public void ensureCoordinatorKnown() {
-        while (coordinatorUnknown()) {
-            RequestFuture<Void> future = sendConsumerMetadataRequest();
-            client.poll(future, requestTimeoutMs);
-
-            if (future.failed())
-                client.awaitMetadataUpdate();
-        }
-    }
-
-    /**
-     * Commit offsets. This call blocks (regardless of commitType) until the coordinator
-     * can receive the commit request. Once the request has been made, however, only the
-     * synchronous commits will wait for a successful response from the coordinator.
-     * @param offsets Offsets to commit.
-     * @param commitType Commit policy
-     * @param callback Callback to be executed when the commit request finishes
-     */
-    public void commitOffsets(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
-        if (commitType == CommitType.ASYNC)
-            commitOffsetsAsync(offsets, callback);
-        else
-            commitOffsetsSync(offsets, callback);
-    }
-
-    private class HeartbeatTask implements DelayedTask {
-
-        public void reset() {
-            // start or restart the heartbeat task to be executed at the next chance
-            long now = time.milliseconds();
-            heartbeat.resetSessionTimeout(now);
-            client.unschedule(this);
-            client.schedule(this, now);
-        }
-
-        @Override
-        public void run(final long now) {
-            if (!subscriptions.partitionsAutoAssigned() ||
-                    subscriptions.partitionAssignmentNeeded() ||
-                    coordinatorUnknown())
-                // no need to send if we're not using auto-assignment or if we are
-                // awaiting a rebalance
-                return;
-
-            if (heartbeat.sessionTimeoutExpired(now)) {
-                // we haven't received a successful heartbeat in one session interval
-                // so mark the coordinator dead
-                coordinatorDead();
-                return;
-            }
-
-            if (!heartbeat.shouldHeartbeat(now)) {
-                // we don't need to heartbeat now, so reschedule for when we do
-                client.schedule(this, now + heartbeat.timeToNextHeartbeat(now));
-            } else {
-                heartbeat.sentHeartbeat(now);
-                RequestFuture<Void> future = sendHeartbeatRequest();
-                future.addListener(new RequestFutureListener<Void>() {
-                    @Override
-                    public void onSuccess(Void value) {
-                        long now = time.milliseconds();
-                        heartbeat.receiveHeartbeat(now);
-                        long nextHeartbeatTime = now + heartbeat.timeToNextHeartbeat(now);
-                        client.schedule(HeartbeatTask.this, nextHeartbeatTime);
-                    }
-
-                    @Override
-                    public void onFailure(RuntimeException e) {
-                        client.schedule(HeartbeatTask.this, time.milliseconds() + retryBackoffMs);
-                    }
-                });
-            }
-        }
-    }
-
-    /**
-     * Send a request to get a new partition assignment. This is a non-blocking call which sends
-     * a JoinGroup request to the coordinator (if it is available). The returned future must
-     * be polled to see if the request completed successfully.
-     * @return A request future whose completion indicates the result of the JoinGroup request.
-     */
-    private RequestFuture<Void> sendJoinGroupRequest() {
-        if (coordinatorUnknown())
-            return RequestFuture.coordinatorNotAvailable();
-
-        // send a join group request to the coordinator
-        List<String> subscribedTopics = new ArrayList<String>(subscriptions.subscribedTopics());
-        log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics);
-
-        JoinGroupRequest request = new JoinGroupRequest(groupId,
-                this.sessionTimeoutMs,
-                subscribedTopics,
-                this.consumerId,
-                this.assignmentStrategy);
-
-        // create the request for the coordinator
-        log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id());
-        return client.send(consumerCoordinator, ApiKeys.JOIN_GROUP, request)
-                .compose(new JoinGroupResponseHandler());
-    }
-
-    private class JoinGroupResponseHandler extends CoordinatorResponseHandler<JoinGroupResponse, Void> {
-
-        @Override
-        public JoinGroupResponse parse(ClientResponse response) {
-            return new JoinGroupResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(JoinGroupResponse joinResponse, RequestFuture<Void> future) {
-            // process the response
-            short errorCode = joinResponse.errorCode();
-
-            if (errorCode == Errors.NONE.code()) {
-                Coordinator.this.consumerId = joinResponse.consumerId();
-                Coordinator.this.generation = joinResponse.generationId();
-
-                // set the flag to refresh last committed offsets
-                subscriptions.needRefreshCommits();
-
-                log.debug("Joined group: {}", joinResponse.toStruct());
-
-                // record re-assignment time
-                sensors.partitionReassignments.record(response.requestLatencyMs());
-
-                // update partition assignment
-                subscriptions.changePartitionAssignment(joinResponse.assignedPartitions());
-                heartbeatTask.reset();
-                future.complete(null);
-            } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) {
-                // reset the consumer id and retry immediately
-                Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-                log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.",
-                        groupId);
-                future.raise(Errors.UNKNOWN_CONSUMER_ID);
-            } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
-                    || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                // re-discover the coordinator and retry with backoff
-                coordinatorDead();
-                log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.",
-                        groupId);
-                future.raise(Errors.forCode(errorCode));
-            } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code()
-                    || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code()
-                    || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) {
-                // log the error and re-throw the exception
-                Errors error = Errors.forCode(errorCode);
-                log.error("Attempt to join group {} failed due to: {}",
-                        groupId, error.exception().getMessage());
-                future.raise(error);
-            } else {
-                // unexpected error, throw the exception
-                future.raise(new KafkaException("Unexpected error in join group response: "
-                        + Errors.forCode(joinResponse.errorCode()).exception().getMessage()));
-            }
-        }
-    }
-
-    private void commitOffsetsAsync(final Map<TopicPartition, Long> offsets, final ConsumerCommitCallback callback) {
-        this.subscriptions.needRefreshCommits();
-        RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
-        if (callback != null) {
-            future.addListener(new RequestFutureListener<Void>() {
-                @Override
-                public void onSuccess(Void value) {
-                    callback.onComplete(offsets, null);
-                }
-
-                @Override
-                public void onFailure(RuntimeException e) {
-                    callback.onComplete(offsets, e);
-                }
-            });
-        }
-    }
-
-    private void commitOffsetsSync(Map<TopicPartition, Long> offsets, ConsumerCommitCallback callback) {
-        while (true) {
-            ensureCoordinatorKnown();
-            ensurePartitionAssignment();
-
-            RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
-            client.poll(future);
-
-            if (future.succeeded()) {
-                if (callback != null)
-                    callback.onComplete(offsets, null);
-                return;
-            }
-
-            if (!future.isRetriable()) {
-                if (callback == null)
-                    throw future.exception();
-                else
-                    callback.onComplete(offsets, future.exception());
-                return;
-            }
-
-            Utils.sleep(retryBackoffMs);
-        }
-    }
-
-    /**
-     * Commit offsets for the specified list of topics and partitions. This is a non-blocking call
-     * which returns a request future that can be polled in the case of a synchronous commit or ignored in the
-     * asynchronous case.
-     *
-     * @param offsets The list of offsets per partition that should be committed.
-     * @return A request future whose value indicates whether the commit was successful or not
-     */
-    private RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, Long> offsets) {
-        if (coordinatorUnknown())
-            return RequestFuture.coordinatorNotAvailable();
-
-        if (offsets.isEmpty())
-            return RequestFuture.voidSuccess();
-
-        // create the offset commit request
-        Map<TopicPartition, OffsetCommitRequest.PartitionData> offsetData;
-        offsetData = new HashMap<TopicPartition, OffsetCommitRequest.PartitionData>(offsets.size());
-        for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet())
-            offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), ""));
-        OffsetCommitRequest req = new OffsetCommitRequest(this.groupId,
-                this.generation,
-                this.consumerId,
-                OffsetCommitRequest.DEFAULT_RETENTION_TIME,
-                offsetData);
-
-        return client.send(consumerCoordinator, ApiKeys.OFFSET_COMMIT, req)
-                .compose(new OffsetCommitResponseHandler(offsets));
-    }
-
-
-    private class OffsetCommitResponseHandler extends CoordinatorResponseHandler<OffsetCommitResponse, Void> {
-
-        private final Map<TopicPartition, Long> offsets;
-
-        public OffsetCommitResponseHandler(Map<TopicPartition, Long> offsets) {
-            this.offsets = offsets;
-        }
-
-        @Override
-        public OffsetCommitResponse parse(ClientResponse response) {
-            return new OffsetCommitResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> future) {
-            sensors.commitLatency.record(response.requestLatencyMs());
-            for (Map.Entry<TopicPartition, Short> entry : commitResponse.responseData().entrySet()) {
-                TopicPartition tp = entry.getKey();
-                long offset = this.offsets.get(tp);
-                short errorCode = entry.getValue();
-                if (errorCode == Errors.NONE.code()) {
-                    log.debug("Committed offset {} for partition {}", offset, tp);
-                    subscriptions.committed(tp, offset);
-                } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
-                        || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                    coordinatorDead();
-                    future.raise(Errors.forCode(errorCode));
-                    return;
-                } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code()
-                        || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) {
-                    // do not need to throw the exception but just log the error
-                    log.error("Error committing partition {} at offset {}: {}",
-                            tp,
-                            offset,
-                            Errors.forCode(errorCode).exception().getMessage());
-                } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
-                        || errorCode == Errors.ILLEGAL_GENERATION.code()) {
-                    // need to re-join group
-                    subscriptions.needReassignment();
-                    future.raise(Errors.forCode(errorCode));
-                    return;
-                } else {
-                    // do not need to throw the exception but just log the error
-                    future.raise(Errors.forCode(errorCode));
-                    log.error("Error committing partition {} at offset {}: {}",
-                            tp,
-                            offset,
-                            Errors.forCode(errorCode).exception().getMessage());
-                }
-            }
-
-            future.complete(null);
-        }
-    }
-
-    /**
-     * Fetch the committed offsets for a set of partitions. This is a non-blocking call. The
-     * returned future can be polled to get the actual offsets returned from the broker.
-     *
-     * @param partitions The set of partitions to get offsets for.
-     * @return A request future containing the committed offsets.
-     */
-    private RequestFuture<Map<TopicPartition, Long>> sendOffsetFetchRequest(Set<TopicPartition> partitions) {
-        if (coordinatorUnknown())
-            return RequestFuture.coordinatorNotAvailable();
-
-        log.debug("Fetching committed offsets for partitions: {}",  Utils.join(partitions, ", "));
-        // construct the request
-        OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList<TopicPartition>(partitions));
-
-        // send the request with a callback
-        return client.send(consumerCoordinator, ApiKeys.OFFSET_FETCH, request)
-                .compose(new OffsetFetchResponseHandler());
-    }
-
-    private class OffsetFetchResponseHandler extends CoordinatorResponseHandler<OffsetFetchResponse, Map<TopicPartition, Long>> {
-
-        @Override
-        public OffsetFetchResponse parse(ClientResponse response) {
-            return new OffsetFetchResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, Long>> future) {
-            Map<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>(response.responseData().size());
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) {
-                TopicPartition tp = entry.getKey();
-                OffsetFetchResponse.PartitionData data = entry.getValue();
-                if (data.hasError()) {
-                    log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode)
-                            .exception()
-                            .getMessage());
-                    if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) {
-                        // just retry
-                        future.raise(Errors.OFFSET_LOAD_IN_PROGRESS);
-                    } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                        // re-discover the coordinator and retry
-                        coordinatorDead();
-                        future.raise(Errors.NOT_COORDINATOR_FOR_CONSUMER);
-                    } else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
-                            || data.errorCode == Errors.ILLEGAL_GENERATION.code()) {
-                        // need to re-join group
-                        subscriptions.needReassignment();
-                        future.raise(Errors.forCode(data.errorCode));
-                    } else {
-                        future.raise(new KafkaException("Unexpected error in fetch offset response: "
-                                + Errors.forCode(data.errorCode).exception().getMessage()));
-                    }
-                    return;
-                } else if (data.offset >= 0) {
-                    // record the position with the offset (-1 indicates no committed offset to fetch)
-                    offsets.put(tp, data.offset);
-                } else {
-                    log.debug("No committed offset for partition " + tp);
-                }
-            }
-
-            future.complete(offsets);
-        }
-    }
-
-    /**
-     * Send a heartbeat request now (visible only for testing).
-     */
-    public RequestFuture<Void> sendHeartbeatRequest() {
-        HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId);
-        return client.send(consumerCoordinator, ApiKeys.HEARTBEAT, req)
-                .compose(new HeartbeatCompletionHandler());
-    }
-
-    public boolean coordinatorUnknown() {
-        return this.consumerCoordinator == null;
-    }
-
-    /**
-     * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to
-     * one of the brokers. The returned future should be polled to get the result of the request.
-     * @return A request future which indicates the completion of the metadata request
-     */
-    private RequestFuture<Void> sendConsumerMetadataRequest() {
-        // initiate the consumer metadata request
-        // find a node to ask about the coordinator
-        Node node = this.client.leastLoadedNode();
-        if (node == null) {
-            // TODO: If there are no brokers left, perhaps we should use the bootstrap set
-            // from configuration?
-            return RequestFuture.noBrokersAvailable();
-        } else {
-            // create a consumer metadata request
-            log.debug("Issuing consumer metadata request to broker {}", node.id());
-            ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId);
-            return client.send(node, ApiKeys.CONSUMER_METADATA, metadataRequest)
-                    .compose(new RequestFutureAdapter<ClientResponse, Void>() {
-                        @Override
-                        public void onSuccess(ClientResponse response, RequestFuture<Void> future) {
-                            handleConsumerMetadataResponse(response, future);
-                        }
-                    });
-        }
-    }
-
-    private void handleConsumerMetadataResponse(ClientResponse resp, RequestFuture<Void> future) {
-        log.debug("Consumer metadata response {}", resp);
-
-        // parse the response to get the coordinator info if it is not disconnected,
-        // otherwise we need to request metadata update
-        if (resp.wasDisconnected()) {
-            future.raise(new DisconnectException());
-        } else if (!coordinatorUnknown()) {
-            // We already found the coordinator, so ignore the request
-            future.complete(null);
-        } else {
-            ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody());
-            // use MAX_VALUE - node.id as the coordinator id to mimic separate connections
-            // for the coordinator in the underlying network client layer
-            // TODO: this needs to be better handled in KAFKA-1935
-            if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) {
-                this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(),
-                        consumerMetadataResponse.node().host(),
-                        consumerMetadataResponse.node().port());
-                heartbeatTask.reset();
-                future.complete(null);
-            } else {
-                future.raise(Errors.forCode(consumerMetadataResponse.errorCode()));
-            }
-        }
-    }
-
-    /**
-     * Mark the current coordinator as dead.
-     */
-    private void coordinatorDead() {
-        if (this.consumerCoordinator != null) {
-            log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id());
-            this.consumerCoordinator = null;
-        }
-    }
-
-    private class HeartbeatCompletionHandler extends CoordinatorResponseHandler<HeartbeatResponse, Void> {
-        @Override
-        public HeartbeatResponse parse(ClientResponse response) {
-            return new HeartbeatResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(HeartbeatResponse heartbeatResponse, RequestFuture<Void> future) {
-            sensors.heartbeatLatency.record(response.requestLatencyMs());
-            short error = heartbeatResponse.errorCode();
-            if (error == Errors.NONE.code()) {
-                log.debug("Received successful heartbeat response.");
-                future.complete(null);
-            } else if (error == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
-                    || error == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead.");
-                coordinatorDead();
-                future.raise(Errors.forCode(error));
-            } else if (error == Errors.ILLEGAL_GENERATION.code()) {
-                log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group.");
-                subscriptions.needReassignment();
-                future.raise(Errors.ILLEGAL_GENERATION);
-            } else if (error == Errors.UNKNOWN_CONSUMER_ID.code()) {
-                log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group.");
-                consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-                subscriptions.needReassignment();
-                future.raise(Errors.UNKNOWN_CONSUMER_ID);
-            } else {
-                future.raise(new KafkaException("Unexpected error in heartbeat response: "
-                        + Errors.forCode(error).exception().getMessage()));
-            }
-        }
-    }
-
-    private abstract class CoordinatorResponseHandler<R, T>
-            extends RequestFutureAdapter<ClientResponse, T> {
-        protected ClientResponse response;
-
-        public abstract R parse(ClientResponse response);
-
-        public abstract void handle(R response, RequestFuture<T> future);
-
-        @Override
-        public void onSuccess(ClientResponse clientResponse, RequestFuture<T> future) {
-            this.response = clientResponse;
-
-            if (clientResponse.wasDisconnected()) {
-                int correlation = response.request().request().header().correlationId();
-                log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected",
-                        response.request(),
-                        correlation,
-                        response.request().request().destination());
-
-                // mark the coordinator as dead
-                coordinatorDead();
-                future.raise(new DisconnectException());
-                return;
-            }
-
-            R response = parse(clientResponse);
-            handle(response, future);
-        }
-
-        @Override
-        public void onFailure(RuntimeException e, RequestFuture<T> future) {
-            if (e instanceof DisconnectException) {
-                log.debug("Coordinator request failed", e);
-                coordinatorDead();
-            }
-            future.raise(e);
-        }
-    }
-
-    public interface RebalanceCallback {
-        void onPartitionsAssigned(Collection<TopicPartition> partitions);
-        void onPartitionsRevoked(Collection<TopicPartition> partitions);
-    }
-
-    private class CoordinatorMetrics {
-        public final Metrics metrics;
-        public final String metricGrpName;
-
-        public final Sensor commitLatency;
-        public final Sensor heartbeatLatency;
-        public final Sensor partitionReassignments;
-
-        public CoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
-            this.metrics = metrics;
-            this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
-
-            this.commitLatency = metrics.sensor("commit-latency");
-            this.commitLatency.add(new MetricName("commit-latency-avg",
-                this.metricGrpName,
-                "The average time taken for a commit request",
-                tags), new Avg());
-            this.commitLatency.add(new MetricName("commit-latency-max",
-                this.metricGrpName,
-                "The max time taken for a commit request",
-                tags), new Max());
-            this.commitLatency.add(new MetricName("commit-rate",
-                this.metricGrpName,
-                "The number of commit calls per second",
-                tags), new Rate(new Count()));
-
-            this.heartbeatLatency = metrics.sensor("heartbeat-latency");
-            this.heartbeatLatency.add(new MetricName("heartbeat-response-time-max",
-                this.metricGrpName,
-                "The max time taken to receive a response to a hearbeat request",
-                tags), new Max());
-            this.heartbeatLatency.add(new MetricName("heartbeat-rate",
-                this.metricGrpName,
-                "The average number of heartbeats per second",
-                tags), new Rate(new Count()));
-
-            this.partitionReassignments = metrics.sensor("reassignment-latency");
-            this.partitionReassignments.add(new MetricName("reassignment-time-avg",
-                this.metricGrpName,
-                "The average time taken for a partition reassignment",
-                tags), new Avg());
-            this.partitionReassignments.add(new MetricName("reassignment-time-max",
-                this.metricGrpName,
-                "The max time taken for a partition reassignment",
-                tags), new Avg());
-            this.partitionReassignments.add(new MetricName("reassignment-rate",
-                this.metricGrpName,
-                "The number of partition reassignments per second",
-                tags), new Rate(new Count()));
-
-            Measurable numParts =
-                new Measurable() {
-                    public double measure(MetricConfig config, long now) {
-                        return subscriptions.assignedPartitions().size();
-                    }
-                };
-            metrics.addMetric(new MetricName("assigned-partitions",
-                this.metricGrpName,
-                "The number of partitions currently assigned to this consumer",
-                tags),
-                numParts);
-
-            Measurable lastHeartbeat =
-                new Measurable() {
-                    public double measure(MetricConfig config, long now) {
-                        return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS);
-                    }
-                };
-            metrics.addMetric(new MetricName("last-heartbeat-seconds-ago",
-                this.metricGrpName,
-                "The number of seconds since the last controller heartbeat",
-                tags),
-                lastHeartbeat);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTask.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTask.java
deleted file mode 100644
index 7792dff..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTask.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-
-package org.apache.flink.kafka_backport.clients.consumer.internals;
-
-
-public interface DelayedTask {
-
-    /**
-     * Execute the task.
-     * @param now current time in milliseconds
-     */
-    void run(long now);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTaskQueue.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTaskQueue.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTaskQueue.java
deleted file mode 100644
index fbd1e19..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/DelayedTaskQueue.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.kafka_backport.clients.consumer.internals;
-
-import java.util.Iterator;
-import java.util.PriorityQueue;
-
-/**
- * Tracks a set of tasks to be executed after a delay.
- */
-public class DelayedTaskQueue {
-
-    private PriorityQueue<Entry> tasks;
-
-    public DelayedTaskQueue() {
-        tasks = new PriorityQueue<Entry>();
-    }
-
-    /**
-     * Schedule a task for execution in the future.
-     *
-     * @param task the task to execute
-     * @param at the time at which to
-     */
-    public void add(DelayedTask task, long at) {
-        tasks.add(new Entry(task, at));
-    }
-
-    /**
-     * Remove a task from the queue if it is present
-     * @param task the task to be removed
-     * @returns true if a task was removed as a result of this call
-     */
-    public boolean remove(DelayedTask task) {
-        boolean wasRemoved = false;
-        Iterator<Entry> iterator = tasks.iterator();
-        while (iterator.hasNext()) {
-            Entry entry = iterator.next();
-            if (entry.task.equals(task)) {
-                iterator.remove();
-                wasRemoved = true;
-            }
-        }
-        return wasRemoved;
-    }
-
-    /**
-     * Get amount of time in milliseconds until the next event. Returns Long.MAX_VALUE if no tasks are scheduled.
-     *
-     * @return the remaining time in milliseconds
-     */
-    public long nextTimeout(long now) {
-        if (tasks.isEmpty())
-            return Long.MAX_VALUE;
-        else
-            return Math.max(tasks.peek().timeout - now, 0);
-    }
-
-    /**
-     * Run any ready tasks.
-     *
-     * @param now the current time
-     */
-    public void poll(long now) {
-        while (!tasks.isEmpty() && tasks.peek().timeout <= now) {
-            Entry entry = tasks.poll();
-            entry.task.run(now);
-        }
-    }
-
-    private static class Entry implements Comparable<Entry> {
-        DelayedTask task;
-        long timeout;
-
-        public Entry(DelayedTask task, long timeout) {
-            this.task = task;
-            this.timeout = timeout;
-        }
-
-        @Override
-        public int compareTo(Entry entry) {
-            return Long.compare(timeout, entry.timeout);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Fetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Fetcher.java
deleted file mode 100644
index 6962a54..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Fetcher.java
+++ /dev/null
@@ -1,506 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-
-package org.apache.flink.kafka_backport.clients.consumer.internals;
-
-import org.apache.flink.kafka_backport.clients.ClientResponse;
-import org.apache.flink.kafka_backport.clients.Metadata;
-import org.apache.flink.kafka_backport.clients.consumer.NoOffsetForPartitionException;
-import org.apache.flink.kafka_backport.clients.consumer.OffsetResetStrategy;
-import org.apache.flink.kafka_backport.common.Node;
-import org.apache.flink.kafka_backport.common.PartitionInfo;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.errors.InvalidMetadataException;
-import org.apache.flink.kafka_backport.common.metrics.Metrics;
-import org.apache.flink.kafka_backport.common.metrics.Sensor;
-import org.apache.flink.kafka_backport.common.metrics.stats.Avg;
-import org.apache.flink.kafka_backport.common.metrics.stats.Rate;
-import org.apache.flink.kafka_backport.common.record.LogEntry;
-import org.apache.flink.kafka_backport.common.record.MemoryRecords;
-import org.apache.flink.kafka_backport.common.requests.FetchRequest;
-import org.apache.flink.kafka_backport.common.requests.FetchResponse;
-import org.apache.flink.kafka_backport.common.serialization.Deserializer;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-import org.apache.flink.kafka_backport.clients.consumer.ConsumerRecord;
-import org.apache.flink.kafka_backport.common.Cluster;
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.apache.flink.kafka_backport.common.errors.DisconnectException;
-import org.apache.flink.kafka_backport.common.metrics.stats.Count;
-import org.apache.flink.kafka_backport.common.metrics.stats.Max;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-import org.apache.flink.kafka_backport.common.requests.ListOffsetRequest;
-import org.apache.flink.kafka_backport.common.requests.ListOffsetResponse;
-import org.apache.flink.kafka_backport.common.utils.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This class manage the fetching process with the brokers.
- */
-public class Fetcher<K, V> {
-    private static final long EARLIEST_OFFSET_TIMESTAMP = -2L;
-    private static final long LATEST_OFFSET_TIMESTAMP = -1L;
-
-    private static final Logger log = LoggerFactory.getLogger(Fetcher.class);
-
-    private final ConsumerNetworkClient client;
-    private final Time time;
-    private final int minBytes;
-    private final int maxWaitMs;
-    private final int fetchSize;
-    private final long retryBackoffMs;
-    private final boolean checkCrcs;
-    private final Metadata metadata;
-    private final FetchManagerMetrics sensors;
-    private final SubscriptionState subscriptions;
-    private final List<PartitionRecords<K, V>> records;
-    private final Deserializer<K> keyDeserializer;
-    private final Deserializer<V> valueDeserializer;
-
-    public Fetcher(ConsumerNetworkClient client,
-                   int minBytes,
-                   int maxWaitMs,
-                   int fetchSize,
-                   boolean checkCrcs,
-                   Deserializer<K> keyDeserializer,
-                   Deserializer<V> valueDeserializer,
-                   Metadata metadata,
-                   SubscriptionState subscriptions,
-                   Metrics metrics,
-                   String metricGrpPrefix,
-                   Map<String, String> metricTags,
-                   Time time,
-                   long retryBackoffMs) {
-
-        this.time = time;
-        this.client = client;
-        this.metadata = metadata;
-        this.subscriptions = subscriptions;
-        this.minBytes = minBytes;
-        this.maxWaitMs = maxWaitMs;
-        this.fetchSize = fetchSize;
-        this.checkCrcs = checkCrcs;
-
-        this.keyDeserializer = keyDeserializer;
-        this.valueDeserializer = valueDeserializer;
-
-        this.records = new LinkedList<PartitionRecords<K, V>>();
-
-        this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags);
-        this.retryBackoffMs = retryBackoffMs;
-    }
-
-    /**
-     * Set-up a fetch request for any node that we have assigned partitions for which doesn't have one.
-     *
-     * @param cluster The current cluster metadata
-     */
-    public void initFetches(Cluster cluster) {
-        for (Map.Entry<Node, FetchRequest> fetchEntry: createFetchRequests(cluster).entrySet()) {
-            final FetchRequest fetch = fetchEntry.getValue();
-            client.send(fetchEntry.getKey(), ApiKeys.FETCH, fetch)
-                    .addListener(new RequestFutureListener<ClientResponse>() {
-                        @Override
-                        public void onSuccess(ClientResponse response) {
-                            handleFetchResponse(response, fetch);
-                        }
-
-                        @Override
-                        public void onFailure(RuntimeException e) {
-                            log.debug("Fetch failed", e);
-                        }
-                    });
-        }
-    }
-
-    /**
-     * Update the fetch positions for the provided partitions.
-     * @param partitions
-     */
-    public void updateFetchPositions(Set<TopicPartition> partitions) {
-        // reset the fetch position to the committed position
-        for (TopicPartition tp : partitions) {
-            // skip if we already have a fetch position
-            if (subscriptions.fetched(tp) != null)
-                continue;
-
-            // TODO: If there are several offsets to reset, we could submit offset requests in parallel
-            if (subscriptions.isOffsetResetNeeded(tp)) {
-                resetOffset(tp);
-            } else if (subscriptions.committed(tp) == null) {
-                // there's no committed position, so we need to reset with the default strategy
-                subscriptions.needOffsetReset(tp);
-                resetOffset(tp);
-            } else {
-                log.debug("Resetting offset for partition {} to the committed offset {}",
-                        tp, subscriptions.committed(tp));
-                subscriptions.seek(tp, subscriptions.committed(tp));
-            }
-        }
-    }
-
-    /**
-     * Reset offsets for the given partition using the offset reset strategy.
-     *
-     * @param partition The given partition that needs reset offset
-     * @throws org.apache.flink.kafka_backport.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined
-     */
-    private void resetOffset(TopicPartition partition) {
-        OffsetResetStrategy strategy = subscriptions.resetStrategy(partition);
-        final long timestamp;
-        if (strategy == OffsetResetStrategy.EARLIEST)
-            timestamp = EARLIEST_OFFSET_TIMESTAMP;
-        else if (strategy == OffsetResetStrategy.LATEST)
-            timestamp = LATEST_OFFSET_TIMESTAMP;
-        else
-            throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined");
-
-        log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase());
-        long offset = listOffset(partition, timestamp);
-        this.subscriptions.seek(partition, offset);
-    }
-
-    /**
-     * Fetch a single offset before the given timestamp for the partition.
-     *
-     * @param partition The partition that needs fetching offset.
-     * @param timestamp The timestamp for fetching offset.
-     * @return The offset of the message that is published before the given timestamp
-     */
-    private long listOffset(TopicPartition partition, long timestamp) {
-        while (true) {
-            RequestFuture<Long> future = sendListOffsetRequest(partition, timestamp);
-            client.poll(future);
-
-            if (future.succeeded())
-                return future.value();
-
-            if (!future.isRetriable())
-                throw future.exception();
-
-            if (future.exception() instanceof InvalidMetadataException)
-                client.awaitMetadataUpdate();
-            else
-                Utils.sleep(retryBackoffMs);
-        }
-    }
-
-    /**
-     * Return the fetched records, empty the record buffer and update the consumed position.
-     *
-     * @return The fetched records per partition
-     */
-    public Map<TopicPartition, List<ConsumerRecord<K, V>>> fetchedRecords() {
-        if (this.subscriptions.partitionAssignmentNeeded()) {
-            return Collections.emptyMap();
-        } else {
-            Map<TopicPartition, List<ConsumerRecord<K, V>>> drained = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
-            for (PartitionRecords<K, V> part : this.records) {
-                Long consumed = subscriptions.consumed(part.partition);
-                if (this.subscriptions.assignedPartitions().contains(part.partition)
-                    && (consumed == null || part.fetchOffset == consumed)) {
-                    List<ConsumerRecord<K, V>> records = drained.get(part.partition);
-                    if (records == null) {
-                        records = part.records;
-                        drained.put(part.partition, records);
-                    } else {
-                        records.addAll(part.records);
-                    }
-                    subscriptions.consumed(part.partition, part.records.get(part.records.size() - 1).offset() + 1);
-                } else {
-                    // these records aren't next in line based on the last consumed position, ignore them
-                    // they must be from an obsolete request
-                    log.debug("Ignoring fetched records for {} at offset {}", part.partition, part.fetchOffset);
-                }
-            }
-            this.records.clear();
-            return drained;
-        }
-    }
-
-    /**
-     * Fetch a single offset before the given timestamp for the partition.
-     *
-     * @param topicPartition The partition that needs fetching offset.
-     * @param timestamp The timestamp for fetching offset.
-     * @return A response which can be polled to obtain the corresponding offset.
-     */
-    private RequestFuture<Long> sendListOffsetRequest(final TopicPartition topicPartition, long timestamp) {
-        Map<TopicPartition, ListOffsetRequest.PartitionData> partitions = new HashMap<TopicPartition, ListOffsetRequest.PartitionData>(1);
-        partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1));
-        PartitionInfo info = metadata.fetch().partition(topicPartition);
-        if (info == null) {
-            metadata.add(topicPartition.topic());
-            log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition);
-            return RequestFuture.staleMetadata();
-        } else if (info.leader() == null) {
-            log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition);
-            return RequestFuture.leaderNotAvailable();
-        } else {
-            Node node = info.leader();
-            ListOffsetRequest request = new ListOffsetRequest(-1, partitions);
-            return client.send(node, ApiKeys.LIST_OFFSETS, request)
-                    .compose(new RequestFutureAdapter<ClientResponse, Long>() {
-                        @Override
-                        public void onSuccess(ClientResponse response, RequestFuture<Long> future) {
-                            handleListOffsetResponse(topicPartition, response, future);
-                        }
-                    });
-        }
-    }
-
-    /**
-     * Callback for the response of the list offset call above.
-     * @param topicPartition The partition that was fetched
-     * @param clientResponse The response from the server.
-     */
-    private void handleListOffsetResponse(TopicPartition topicPartition,
-                                          ClientResponse clientResponse,
-                                          RequestFuture<Long> future) {
-        if (clientResponse.wasDisconnected()) {
-            future.raise(new DisconnectException());
-        } else {
-            ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody());
-            short errorCode = lor.responseData().get(topicPartition).errorCode;
-            if (errorCode == Errors.NONE.code()) {
-                List<Long> offsets = lor.responseData().get(topicPartition).offsets;
-                if (offsets.size() != 1)
-                    throw new IllegalStateException("This should not happen.");
-                long offset = offsets.get(0);
-                log.debug("Fetched offset {} for partition {}", offset, topicPartition);
-
-                future.complete(offset);
-            } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code()
-                    || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) {
-                log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.",
-                        topicPartition);
-                future.raise(Errors.forCode(errorCode));
-            } else {
-                log.error("Attempt to fetch offsets for partition {} failed due to: {}",
-                        topicPartition, Errors.forCode(errorCode).exception().getMessage());
-                future.raise(new StaleMetadataException());
-            }
-        }
-    }
-
-    /**
-     * Create fetch requests for all nodes for which we have assigned partitions
-     * that have no existing requests in flight.
-     */
-    private Map<Node, FetchRequest> createFetchRequests(Cluster cluster) {
-        // create the fetch info
-        Map<Node, Map<TopicPartition, FetchRequest.PartitionData>> fetchable = new HashMap<Node, Map<TopicPartition, FetchRequest.PartitionData>>();
-        for (TopicPartition partition : subscriptions.assignedPartitions()) {
-            Node node = cluster.leaderFor(partition);
-            if (node == null) {
-                metadata.requestUpdate();
-            } else if (this.client.pendingRequestCount(node) == 0) {
-                // if there is a leader and no in-flight requests, issue a new fetch
-                Map<TopicPartition, FetchRequest.PartitionData> fetch = fetchable.get(node);
-                if (fetch == null) {
-                    fetch = new HashMap<TopicPartition, FetchRequest.PartitionData>();
-                    fetchable.put(node, fetch);
-                }
-                long offset = this.subscriptions.fetched(partition);
-                fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize));
-            }
-        }
-
-        // create the fetches
-        Map<Node, FetchRequest> requests = new HashMap<Node, FetchRequest>();
-        for (Map.Entry<Node, Map<TopicPartition, FetchRequest.PartitionData>> entry : fetchable.entrySet()) {
-            Node node = entry.getKey();
-            FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue());
-            requests.put(node, fetch);
-        }
-        return requests;
-    }
-
-    /**
-     * The callback for fetch completion
-     */
-    private void handleFetchResponse(ClientResponse resp, FetchRequest request) {
-        if (resp.wasDisconnected()) {
-            int correlation = resp.request().request().header().correlationId();
-            log.debug("Cancelled fetch request {} with correlation id {} due to node {} being disconnected",
-                resp.request(), correlation, resp.request().request().destination());
-        } else {
-            int totalBytes = 0;
-            int totalCount = 0;
-            FetchResponse response = new FetchResponse(resp.responseBody());
-            for (Map.Entry<TopicPartition, FetchResponse.PartitionData> entry : response.responseData().entrySet()) {
-                TopicPartition tp = entry.getKey();
-                FetchResponse.PartitionData partition = entry.getValue();
-                if (!subscriptions.assignedPartitions().contains(tp)) {
-                    log.debug("Ignoring fetched data for partition {} which is no longer assigned.", tp);
-                } else if (partition.errorCode == Errors.NONE.code()) {
-                    int bytes = 0;
-                    ByteBuffer buffer = partition.recordSet;
-                    MemoryRecords records = MemoryRecords.readableRecords(buffer);
-                    long fetchOffset = request.fetchData().get(tp).offset;
-                    List<ConsumerRecord<K, V>> parsed = new ArrayList<ConsumerRecord<K, V>>();
-                    for (LogEntry logEntry : records) {
-                        parsed.add(parseRecord(tp, logEntry));
-                        bytes += logEntry.size();
-                    }
-                    if (parsed.size() > 0) {
-                        ConsumerRecord<K, V> record = parsed.get(parsed.size() - 1);
-                        this.subscriptions.fetched(tp, record.offset() + 1);
-                        this.records.add(new PartitionRecords<K, V>(fetchOffset, tp, parsed));
-                        this.sensors.recordsFetchLag.record(partition.highWatermark - record.offset());
-                    }
-                    this.sensors.recordTopicFetchMetrics(tp.topic(), bytes, parsed.size());
-                    totalBytes += bytes;
-                    totalCount += parsed.size();
-                } else if (partition.errorCode == Errors.NOT_LEADER_FOR_PARTITION.code()
-                    || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) {
-                    this.metadata.requestUpdate();
-                } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) {
-                    // TODO: this could be optimized by grouping all out-of-range partitions
-                    log.info("Fetch offset {} is out of range, resetting offset", subscriptions.fetched(tp));
-                    subscriptions.needOffsetReset(tp);
-                } else if (partition.errorCode == Errors.UNKNOWN.code()) {
-                    log.warn("Unknown error fetching data for topic-partition {}", tp);
-                } else {
-                    throw new IllegalStateException("Unexpected error code " + partition.errorCode + " while fetching data");
-                }
-            }
-            this.sensors.bytesFetched.record(totalBytes);
-            this.sensors.recordsFetched.record(totalCount);
-        }
-        this.sensors.fetchLatency.record(resp.requestLatencyMs());
-    }
-
-    /**
-     * Parse the record entry, deserializing the key / value fields if necessary
-     */
-    private ConsumerRecord<K, V> parseRecord(TopicPartition partition, LogEntry logEntry) {
-        if (this.checkCrcs)
-            logEntry.record().ensureValid();
-
-        long offset = logEntry.offset();
-        ByteBuffer keyBytes = logEntry.record().key();
-        K key = keyBytes == null ? null : this.keyDeserializer.deserialize(partition.topic(), Utils.toArray(keyBytes));
-        ByteBuffer valueBytes = logEntry.record().value();
-        V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), Utils.toArray(valueBytes));
-
-        return new ConsumerRecord<K, V>(partition.topic(), partition.partition(), offset, key, value);
-    }
-
-    private static class PartitionRecords<K, V> {
-        public long fetchOffset;
-        public TopicPartition partition;
-        public List<ConsumerRecord<K, V>> records;
-
-        public PartitionRecords(long fetchOffset, TopicPartition partition, List<ConsumerRecord<K, V>> records) {
-            this.fetchOffset = fetchOffset;
-            this.partition = partition;
-            this.records = records;
-        }
-    }
-
-    private class FetchManagerMetrics {
-        public final Metrics metrics;
-        public final String metricGrpName;
-
-        public final Sensor bytesFetched;
-        public final Sensor recordsFetched;
-        public final Sensor fetchLatency;
-        public final Sensor recordsFetchLag;
-
-
-        public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
-            this.metrics = metrics;
-            this.metricGrpName = metricGrpPrefix + "-fetch-manager-metrics";
-
-            this.bytesFetched = metrics.sensor("bytes-fetched");
-            this.bytesFetched.add(new MetricName("fetch-size-avg",
-                this.metricGrpName,
-                "The average number of bytes fetched per request",
-                tags), new Avg());
-            this.bytesFetched.add(new MetricName("fetch-size-max",
-                this.metricGrpName,
-                "The maximum number of bytes fetched per request",
-                tags), new Max());
-            this.bytesFetched.add(new MetricName("bytes-consumed-rate",
-                this.metricGrpName,
-                "The average number of bytes consumed per second",
-                tags), new Rate());
-
-            this.recordsFetched = metrics.sensor("records-fetched");
-            this.recordsFetched.add(new MetricName("records-per-request-avg",
-                this.metricGrpName,
-                "The average number of records in each request",
-                tags), new Avg());
-            this.recordsFetched.add(new MetricName("records-consumed-rate",
-                this.metricGrpName,
-                "The average number of records consumed per second",
-                tags), new Rate());
-
-            this.fetchLatency = metrics.sensor("fetch-latency");
-            this.fetchLatency.add(new MetricName("fetch-latency-avg",
-                this.metricGrpName,
-                "The average time taken for a fetch request.",
-                tags), new Avg());
-            this.fetchLatency.add(new MetricName("fetch-latency-max",
-                this.metricGrpName,
-                "The max time taken for any fetch request.",
-                tags), new Max());
-            this.fetchLatency.add(new MetricName("fetch-rate",
-                this.metricGrpName,
-                "The number of fetch requests per second.",
-                tags), new Rate(new Count()));
-
-            this.recordsFetchLag = metrics.sensor("records-lag");
-            this.recordsFetchLag.add(new MetricName("records-lag-max",
-                this.metricGrpName,
-                "The maximum lag in terms of number of records for any partition in this window",
-                tags), new Max());
-        }
-
-        public void recordTopicFetchMetrics(String topic, int bytes, int records) {
-            // record bytes fetched
-            String name = "topic." + topic + ".bytes-fetched";
-            Sensor bytesFetched = this.metrics.getSensor(name);
-            if (bytesFetched == null)
-                bytesFetched = this.metrics.sensor(name);
-            bytesFetched.record(bytes);
-
-            // record records fetched
-            name = "topic." + topic + ".records-fetched";
-            Sensor recordsFetched = this.metrics.getSensor(name);
-            if (recordsFetched == null)
-                recordsFetched = this.metrics.sensor(name);
-            recordsFetched.record(records);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Heartbeat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Heartbeat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Heartbeat.java
deleted file mode 100644
index f412897..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/Heartbeat.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.kafka_backport.clients.consumer.internals;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A helper class for managing the heartbeat to the coordinator
- */
-public final class Heartbeat {
-    
-    /* The number of heartbeats to attempt to complete per session timeout interval.
-     * so, e.g., with a session timeout of 3 seconds we would attempt a heartbeat
-     * once per second.
-     */
-    public final static int HEARTBEATS_PER_SESSION_INTERVAL = 3;
-
-    private final long timeout;
-    private long lastHeartbeatSend;
-    private long lastHeartbeatReceive;
-    private long lastSessionReset;
-
-    public Heartbeat(long timeout, long now) {
-        this.timeout = timeout;
-        this.lastSessionReset = now;
-    }
-
-    public void sentHeartbeat(long now) {
-        this.lastHeartbeatSend = now;
-    }
-
-    public void receiveHeartbeat(long now) {
-        this.lastHeartbeatReceive = now;
-    }
-
-    public boolean shouldHeartbeat(long now) {
-        return timeToNextHeartbeat(now) == 0;
-    }
-    
-    public long lastHeartbeatSend() {
-        return this.lastHeartbeatSend;
-    }
-
-    public long timeToNextHeartbeat(long now) {
-        long timeSinceLastHeartbeat = now - Math.max(lastHeartbeatSend, lastSessionReset);
-
-        long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL;
-        if (timeSinceLastHeartbeat > hbInterval)
-            return 0;
-        else
-            return hbInterval - timeSinceLastHeartbeat;
-    }
-
-    public boolean sessionTimeoutExpired(long now) {
-        return now - Math.max(lastSessionReset, lastHeartbeatReceive) > timeout;
-    }
-
-    public long interval() {
-        return timeout / HEARTBEATS_PER_SESSION_INTERVAL;
-    }
-
-    public void resetSessionTimeout(long now) {
-        this.lastSessionReset = now;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoAvailableBrokersException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoAvailableBrokersException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoAvailableBrokersException.java
deleted file mode 100644
index 421c64e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoAvailableBrokersException.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.kafka_backport.clients.consumer.internals;
-
-import org.apache.flink.kafka_backport.common.errors.InvalidMetadataException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * No brokers were available to complete a request.
- */
-public class NoAvailableBrokersException extends InvalidMetadataException {
-    private static final long serialVersionUID = 1L;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoOpConsumerRebalanceCallback.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
deleted file mode 100644
index 6da31dd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/NoOpConsumerRebalanceCallback.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.kafka_backport.clients.consumer.internals;
-
-import org.apache.flink.kafka_backport.clients.consumer.ConsumerRebalanceCallback;
-import org.apache.flink.kafka_backport.clients.consumer.Consumer;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-
-import java.util.Collection;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class NoOpConsumerRebalanceCallback implements ConsumerRebalanceCallback {
-
-    @Override
-    public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {}
-
-    @Override
-    public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFuture.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFuture.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFuture.java
deleted file mode 100644
index 7b6edc3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFuture.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer.internals;
-
-import org.apache.flink.kafka_backport.common.errors.RetriableException;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-
-import java.util.ArrayList;
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Result of an asynchronous request from {@link ConsumerNetworkClient}. Use {@link ConsumerNetworkClient#poll(long)}
- * (and variants) to finish a request future. Use {@link #isDone()} to check if the future is complete, and
- * {@link #succeeded()} to check if the request completed successfully. Typical usage might look like this:
- *
- * <pre>
- *     RequestFuture<ClientResponse> future = client.send(api, request);
- *     client.poll(future);
- *
- *     if (future.succeeded()) {
- *         ClientResponse response = future.value();
- *         // Handle response
- *     } else {
- *         throw future.exception();
- *     }
- * </pre>
- *
- * @param <T> Return type of the result (Can be Void if there is no response)
- */
-public class RequestFuture<T> {
-
-    private boolean isDone = false;
-    private T value;
-    private RuntimeException exception;
-    private List<RequestFutureListener<T>> listeners = new ArrayList<RequestFutureListener<T>>();
-
-
-    /**
-     * Check whether the response is ready to be handled
-     * @return true if the response is ready, false otherwise
-     */
-    public boolean isDone() {
-        return isDone;
-    }
-
-    /**
-     * Get the value corresponding to this request (only available if the request succeeded)
-     * @return the value if it exists or null
-     */
-    public T value() {
-        return value;
-    }
-
-    /**
-     * Check if the request succeeded;
-     * @return true if the request completed and was successful
-     */
-    public boolean succeeded() {
-        return isDone && exception == null;
-    }
-
-    /**
-     * Check if the request failed.
-     * @return true if the request completed with a failure
-     */
-    public boolean failed() {
-        return isDone && exception != null;
-    }
-
-    /**
-     * Check if the request is retriable (convenience method for checking if
-     * the exception is an instance of {@link org.apache.flink.kafka_backport.common.errors.RetriableException}.
-     * @return true if it is retriable, false otherwise
-     */
-    public boolean isRetriable() {
-        return exception instanceof RetriableException;
-    }
-
-    /**
-     * Get the exception from a failed result (only available if the request failed)
-     * @return The exception if it exists or null
-     */
-    public RuntimeException exception() {
-        return exception;
-    }
-
-    /**
-     * Complete the request successfully. After this call, {@link #succeeded()} will return true
-     * and the value can be obtained through {@link #value()}.
-     * @param value corresponding value (or null if there is none)
-     */
-    public void complete(T value) {
-        this.value = value;
-        this.isDone = true;
-        fireSuccess();
-    }
-
-    /**
-     * Raise an exception. The request will be marked as failed, and the caller can either
-     * handle the exception or throw it.
-     * @param e corresponding exception to be passed to caller
-     */
-    public void raise(RuntimeException e) {
-        this.exception = e;
-        this.isDone = true;
-        fireFailure();
-    }
-
-    /**
-     * Raise an error. The request will be marked as failed.
-     * @param error corresponding error to be passed to caller
-     */
-    public void raise(Errors error) {
-        raise(error.exception());
-    }
-
-    private void fireSuccess() {
-        for (RequestFutureListener listener: listeners)
-            listener.onSuccess(value);
-    }
-
-    private void fireFailure() {
-        for (RequestFutureListener listener: listeners)
-            listener.onFailure(exception);
-    }
-
-    /**
-     * Add a listener which will be notified when the future completes
-     * @param listener
-     */
-    public void addListener(RequestFutureListener<T> listener) {
-        if (isDone) {
-            if (exception != null)
-                listener.onFailure(exception);
-            else
-                listener.onSuccess(value);
-        } else {
-            this.listeners.add(listener);
-        }
-    }
-
-    /**
-     * Convert from a request future of one type to another type
-     * @param adapter The adapter which does the conversion
-     * @param <S> The type of the future adapted to
-     * @return The new future
-     */
-    public <S> RequestFuture<S> compose(final RequestFutureAdapter<T, S> adapter) {
-        final RequestFuture<S> adapted = new RequestFuture<S>();
-        addListener(new RequestFutureListener<T>() {
-            @Override
-            public void onSuccess(T value) {
-                adapter.onSuccess(value, adapted);
-            }
-
-            @Override
-            public void onFailure(RuntimeException e) {
-                adapter.onFailure(e, adapted);
-            }
-        });
-        return adapted;
-    }
-
-    public static <T> RequestFuture<T> failure(RuntimeException e) {
-        RequestFuture<T> future = new RequestFuture<T>();
-        future.raise(e);
-        return future;
-    }
-
-    public static RequestFuture<Void> voidSuccess() {
-        RequestFuture<Void> future = new RequestFuture<Void>();
-        future.complete(null);
-        return future;
-    }
-
-    public static <T> RequestFuture<T> coordinatorNotAvailable() {
-        return failure(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception());
-    }
-
-    public static <T> RequestFuture<T> leaderNotAvailable() {
-        return failure(Errors.LEADER_NOT_AVAILABLE.exception());
-    }
-
-    public static <T> RequestFuture<T> noBrokersAvailable() {
-        return failure(new NoAvailableBrokersException());
-    }
-
-    public static <T> RequestFuture<T> staleMetadata() {
-        return failure(new StaleMetadataException());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureAdapter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureAdapter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureAdapter.java
deleted file mode 100644
index b34c2da..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureAdapter.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.kafka_backport.clients.consumer.internals;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Adapt from a request future of one type to another.
- *
- * @param <F> Type to adapt from
- * @param <T> Type to adapt to
- */
-public abstract class RequestFutureAdapter<F, T> {
-
-    public abstract void onSuccess(F value, RequestFuture<T> future);
-
-    public void onFailure(RuntimeException e, RequestFuture<T> future) {
-        future.raise(e);
-    }
-}


[41/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Compressor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Compressor.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Compressor.java
deleted file mode 100644
index b1b537c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Compressor.java
+++ /dev/null
@@ -1,279 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.record;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.zip.GZIPInputStream;
-import java.util.zip.GZIPOutputStream;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class Compressor {
-
-    static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f;
-    static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f;
-    static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024;
-
-    private static final float[] TYPE_TO_RATE;
-
-    static {
-        int maxTypeId = -1;
-        for (CompressionType type : CompressionType.values())
-            maxTypeId = Math.max(maxTypeId, type.id);
-        TYPE_TO_RATE = new float[maxTypeId + 1];
-        for (CompressionType type : CompressionType.values()) {
-            TYPE_TO_RATE[type.id] = type.rate;
-        }
-    }
-
-    private final CompressionType type;
-    private final DataOutputStream appendStream;
-    private final ByteBufferOutputStream bufferStream;
-    private final int initPos;
-
-    public long writtenUncompressed;
-    public long numRecords;
-
-    public Compressor(ByteBuffer buffer, CompressionType type, int blockSize) {
-        this.type = type;
-        this.initPos = buffer.position();
-
-        this.numRecords = 0;
-        this.writtenUncompressed = 0;
-
-        if (type != CompressionType.NONE) {
-            // for compressed records, leave space for the header and the shallow message metadata
-            // and move the starting position to the value payload offset
-            buffer.position(initPos + Records.LOG_OVERHEAD + Record.RECORD_OVERHEAD);
-        }
-
-        // create the stream
-        bufferStream = new ByteBufferOutputStream(buffer);
-        appendStream = wrapForOutput(bufferStream, type, blockSize);
-    }
-
-    public Compressor(ByteBuffer buffer, CompressionType type) {
-        this(buffer, type, COMPRESSION_DEFAULT_BUFFER_SIZE);
-    }
-
-    public ByteBuffer buffer() {
-        return bufferStream.buffer();
-    }
-    
-    public double compressionRate() {
-        ByteBuffer buffer = bufferStream.buffer();
-        if (this.writtenUncompressed == 0)
-            return 1.0;
-        else
-            return (double) buffer.position() / this.writtenUncompressed;
-    }
-
-    public void close() {
-        try {
-            appendStream.close();
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-
-        if (type != CompressionType.NONE) {
-            ByteBuffer buffer = bufferStream.buffer();
-            int pos = buffer.position();
-            // write the header, for the end offset write as number of records - 1
-            buffer.position(initPos);
-            buffer.putLong(numRecords - 1);
-            buffer.putInt(pos - initPos - Records.LOG_OVERHEAD);
-            // write the shallow message (the crc and value size are not correct yet)
-            Record.write(buffer, null, null, type, 0, -1);
-            // compute the fill the value size
-            int valueSize = pos - initPos - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD;
-            buffer.putInt(initPos + Records.LOG_OVERHEAD + Record.KEY_OFFSET, valueSize);
-            // compute and fill the crc at the beginning of the message
-            long crc = Record.computeChecksum(buffer,
-                    initPos + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET,
-                    pos - initPos - Records.LOG_OVERHEAD - Record.MAGIC_OFFSET);
-            Utils.writeUnsignedInt(buffer, initPos + Records.LOG_OVERHEAD + Record.CRC_OFFSET, crc);
-            // reset the position
-            buffer.position(pos);
-
-            // update the compression ratio
-            float compressionRate = (float) buffer.position() / this.writtenUncompressed;
-            TYPE_TO_RATE[type.id] = TYPE_TO_RATE[type.id] * COMPRESSION_RATE_DAMPING_FACTOR +
-                compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR);
-        }
-    }
-
-    // Note that for all the write operations below, IO exceptions should
-    // never be thrown since the underlying ByteBufferOutputStream does not throw IOException;
-    // therefore upon encountering this issue we just close the append stream.
-
-    public void putLong(final long value) {
-        try {
-            appendStream.writeLong(value);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void putInt(final int value) {
-        try {
-            appendStream.writeInt(value);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void put(final ByteBuffer buffer) {
-        try {
-            appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit());
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void putByte(final byte value) {
-        try {
-            appendStream.write(value);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void put(final byte[] bytes, final int offset, final int len) {
-        try {
-            appendStream.write(bytes, offset, len);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void putRecord(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        // put a record as un-compressed into the underlying stream
-        long crc = Record.computeChecksum(key, value, type, valueOffset, valueSize);
-        byte attributes = Record.computeAttributes(type);
-        putRecord(crc, attributes, key, value, valueOffset, valueSize);
-    }
-
-    public void putRecord(byte[] key, byte[] value) {
-        putRecord(key, value, CompressionType.NONE, 0, -1);
-    }
-
-    private void putRecord(final long crc, final byte attributes, final byte[] key, final byte[] value, final int valueOffset, final int valueSize) {
-        Record.write(this, crc, attributes, key, value, valueOffset, valueSize);
-    }
-
-    public void recordWritten(int size) {
-        numRecords += 1;
-        writtenUncompressed += size;
-    }
-
-    public long estimatedBytesWritten() {
-        if (type == CompressionType.NONE) {
-            return bufferStream.buffer().position();
-        } else {
-            // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
-            return (long) (writtenUncompressed * TYPE_TO_RATE[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR);
-        }
-    }
-
-    // the following two functions also need to be public since they are used in MemoryRecords.iteration
-
-    static public DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, int bufferSize) {
-        try {
-            switch (type) {
-                case NONE:
-                    return new DataOutputStream(buffer);
-                case GZIP:
-                    return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize));
-                case SNAPPY:
-                    // dynamically load the snappy class to avoid runtime dependency
-                    // on snappy if we are not using it
-                    try {
-                        Class<?> outputStreamClass = Class.forName("org.xerial.snappy.SnappyOutputStream");
-                        OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class, Integer.TYPE)
-                            .newInstance(buffer, bufferSize);
-                        return new DataOutputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                case LZ4:
-                    try {
-                        Class<?> outputStreamClass = Class.forName("org.apache.KafkaLZ4BlockOutputStream");
-                        OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class)
-                            .newInstance(buffer);
-                        return new DataOutputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                default:
-                    throw new IllegalArgumentException("Unknown compression type: " + type);
-            }
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-    }
-
-    static public DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type) {
-        try {
-            switch (type) {
-                case NONE:
-                    return new DataInputStream(buffer);
-                case GZIP:
-                    return new DataInputStream(new GZIPInputStream(buffer));
-                case SNAPPY:
-                    // dynamically load the snappy class to avoid runtime dependency
-                    // on snappy if we are not using it
-                    try {
-                        Class<?> inputStreamClass = Class.forName("org.xerial.snappy.SnappyInputStream");
-                        InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class)
-                            .newInstance(buffer);
-                        return new DataInputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                case LZ4:
-                    // dynamically load LZ4 class to avoid runtime dependency
-                    try {
-                        Class<?> inputStreamClass = Class.forName("org.apache.KafkaLZ4BlockInputStream");
-                        InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class)
-                            .newInstance(buffer);
-                        return new DataInputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                default:
-                    throw new IllegalArgumentException("Unknown compression type: " + type);
-            }
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/InvalidRecordException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/InvalidRecordException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/InvalidRecordException.java
deleted file mode 100644
index 38c26ed..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/InvalidRecordException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.record;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class InvalidRecordException extends RuntimeException {
-
-    private static final long serialVersionUID = 1;
-
-    public InvalidRecordException(String s) {
-        super(s);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockInputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockInputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockInputStream.java
deleted file mode 100644
index 1049787..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockInputStream.java
+++ /dev/null
@@ -1,236 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * 
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.kafka_backport.common.record;
-
-import net.jpountz.lz4.LZ4Exception;
-import net.jpountz.lz4.LZ4Factory;
-import net.jpountz.lz4.LZ4SafeDecompressor;
-import net.jpountz.xxhash.XXHash32;
-import net.jpountz.xxhash.XXHashFactory;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-
-import java.io.FilterInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A partial implementation of the v1.4.1 LZ4 Frame format.
- * 
- * @see <a href="https://docs.google.com/document/d/1Tdxmn5_2e5p1y4PtXkatLndWVb0R8QARJFe6JI4Keuo/edit">LZ4 Framing
- *      Format Spec</a>
- */
-public final class KafkaLZ4BlockInputStream extends FilterInputStream {
-
-    public static final String PREMATURE_EOS = "Stream ended prematurely";
-    public static final String NOT_SUPPORTED = "Stream unsupported";
-    public static final String BLOCK_HASH_MISMATCH = "Block checksum mismatch";
-    public static final String DESCRIPTOR_HASH_MISMATCH = "Stream frame descriptor corrupted";
-
-    private final LZ4SafeDecompressor decompressor;
-    private final XXHash32 checksum;
-    private final byte[] buffer;
-    private final byte[] compressedBuffer;
-    private final int maxBlockSize;
-    private KafkaLZ4BlockOutputStream.FLG flg;
-    private KafkaLZ4BlockOutputStream.BD bd;
-    private int bufferOffset;
-    private int bufferSize;
-    private boolean finished;
-
-    /**
-     * Create a new {@link InputStream} that will decompress data using the LZ4 algorithm.
-     * 
-     * @param in The stream to decompress
-     * @throws IOException
-     */
-    public KafkaLZ4BlockInputStream(InputStream in) throws IOException {
-        super(in);
-        decompressor = LZ4Factory.fastestInstance().safeDecompressor();
-        checksum = XXHashFactory.fastestInstance().hash32();
-        readHeader();
-        maxBlockSize = bd.getBlockMaximumSize();
-        buffer = new byte[maxBlockSize];
-        compressedBuffer = new byte[maxBlockSize];
-        bufferOffset = 0;
-        bufferSize = 0;
-        finished = false;
-    }
-
-    /**
-     * Reads the magic number and frame descriptor from the underlying {@link InputStream}.
-     * 
-     * @throws IOException
-     */
-    private void readHeader() throws IOException {
-        byte[] header = new byte[KafkaLZ4BlockOutputStream.LZ4_MAX_HEADER_LENGTH];
-
-        // read first 6 bytes into buffer to check magic and FLG/BD descriptor flags
-        bufferOffset = 6;
-        if (in.read(header, 0, bufferOffset) != bufferOffset) {
-            throw new IOException(PREMATURE_EOS);
-        }
-
-        if (KafkaLZ4BlockOutputStream.MAGIC != Utils.readUnsignedIntLE(header, bufferOffset - 6)) {
-            throw new IOException(NOT_SUPPORTED);
-        }
-        flg = KafkaLZ4BlockOutputStream.FLG.fromByte(header[bufferOffset - 2]);
-        bd = KafkaLZ4BlockOutputStream.BD.fromByte(header[bufferOffset - 1]);
-        // TODO read uncompressed content size, update flg.validate()
-        // TODO read dictionary id, update flg.validate()
-
-        // check stream descriptor hash
-        byte hash = (byte) ((checksum.hash(header, 0, bufferOffset, 0) >> 8) & 0xFF);
-        header[bufferOffset++] = (byte) in.read();
-        if (hash != header[bufferOffset - 1]) {
-            throw new IOException(DESCRIPTOR_HASH_MISMATCH);
-        }
-    }
-
-    /**
-     * Decompresses (if necessary) buffered data, optionally computes and validates a XXHash32 checksum, and writes the
-     * result to a buffer.
-     * 
-     * @throws IOException
-     */
-    private void readBlock() throws IOException {
-        int blockSize = Utils.readUnsignedIntLE(in);
-
-        // Check for EndMark
-        if (blockSize == 0) {
-            finished = true;
-            // TODO implement content checksum, update flg.validate()
-            return;
-        } else if (blockSize > maxBlockSize) {
-            throw new IOException(String.format("Block size %s exceeded max: %s", blockSize, maxBlockSize));
-        }
-
-        boolean compressed = (blockSize & KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK) == 0;
-        byte[] bufferToRead;
-        if (compressed) {
-            bufferToRead = compressedBuffer;
-        } else {
-            blockSize &= ~KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK;
-            bufferToRead = buffer;
-            bufferSize = blockSize;
-        }
-
-        if (in.read(bufferToRead, 0, blockSize) != blockSize) {
-            throw new IOException(PREMATURE_EOS);
-        }
-
-        // verify checksum
-        if (flg.isBlockChecksumSet() && Utils.readUnsignedIntLE(in) != checksum.hash(bufferToRead, 0, blockSize, 0)) {
-            throw new IOException(BLOCK_HASH_MISMATCH);
-        }
-
-        if (compressed) {
-            try {
-                bufferSize = decompressor.decompress(compressedBuffer, 0, blockSize, buffer, 0, maxBlockSize);
-            } catch (LZ4Exception e) {
-                throw new IOException(e);
-            }
-        }
-
-        bufferOffset = 0;
-    }
-
-    @Override
-    public int read() throws IOException {
-        if (finished) {
-            return -1;
-        }
-        if (available() == 0) {
-            readBlock();
-        }
-        if (finished) {
-            return -1;
-        }
-        int value = buffer[bufferOffset++] & 0xFF;
-
-        return value;
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        net.jpountz.util.Utils.checkRange(b, off, len);
-        if (finished) {
-            return -1;
-        }
-        if (available() == 0) {
-            readBlock();
-        }
-        if (finished) {
-            return -1;
-        }
-        len = Math.min(len, available());
-        System.arraycopy(buffer, bufferOffset, b, off, len);
-        bufferOffset += len;
-        return len;
-    }
-
-    @Override
-    public long skip(long n) throws IOException {
-        if (finished) {
-            return 0;
-        }
-        if (available() == 0) {
-            readBlock();
-        }
-        if (finished) {
-            return 0;
-        }
-        n = Math.min(n, available());
-        bufferOffset += n;
-        return n;
-    }
-
-    @Override
-    public int available() throws IOException {
-        return bufferSize - bufferOffset;
-    }
-
-    @Override
-    public void close() throws IOException {
-        in.close();
-    }
-
-    @Override
-    public synchronized void mark(int readlimit) {
-        throw new RuntimeException("mark not supported");
-    }
-
-    @Override
-    public synchronized void reset() throws IOException {
-        throw new RuntimeException("reset not supported");
-    }
-
-    @Override
-    public boolean markSupported() {
-        return false;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockOutputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockOutputStream.java
deleted file mode 100644
index 3007c42..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockOutputStream.java
+++ /dev/null
@@ -1,400 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * 
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.kafka_backport.common.record;
-
-import net.jpountz.lz4.LZ4Compressor;
-import net.jpountz.lz4.LZ4Factory;
-import net.jpountz.xxhash.XXHash32;
-import net.jpountz.xxhash.XXHashFactory;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A partial implementation of the v1.4.1 LZ4 Frame format.
- * 
- * @see <a href="https://docs.google.com/document/d/1Tdxmn5_2e5p1y4PtXkatLndWVb0R8QARJFe6JI4Keuo/edit">LZ4 Framing
- *      Format Spec</a>
- */
-public final class KafkaLZ4BlockOutputStream extends FilterOutputStream {
-
-    public static final int MAGIC = 0x184D2204;
-    public static final int LZ4_MAX_HEADER_LENGTH = 19;
-    public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000;
-
-    public static final String CLOSED_STREAM = "The stream is already closed";
-
-    public static final int BLOCKSIZE_64KB = 4;
-    public static final int BLOCKSIZE_256KB = 5;
-    public static final int BLOCKSIZE_1MB = 6;
-    public static final int BLOCKSIZE_4MB = 7;
-
-    private final LZ4Compressor compressor;
-    private final XXHash32 checksum;
-    private final FLG flg;
-    private final BD bd;
-    private final byte[] buffer;
-    private final byte[] compressedBuffer;
-    private final int maxBlockSize;
-    private int bufferOffset;
-    private boolean finished;
-
-    /**
-     * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
-     * 
-     * @param out The output stream to compress
-     * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other
-     *            values will generate an exception
-     * @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for
-     *            every block of data
-     * @throws IOException
-     */
-    public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum) throws IOException {
-        super(out);
-        compressor = LZ4Factory.fastestInstance().fastCompressor();
-        checksum = XXHashFactory.fastestInstance().hash32();
-        bd = new BD(blockSize);
-        flg = new FLG(blockChecksum);
-        bufferOffset = 0;
-        maxBlockSize = bd.getBlockMaximumSize();
-        buffer = new byte[maxBlockSize];
-        compressedBuffer = new byte[compressor.maxCompressedLength(maxBlockSize)];
-        finished = false;
-        writeHeader();
-    }
-
-    /**
-     * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
-     * 
-     * @param out The stream to compress
-     * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other
-     *            values will generate an exception
-     * @throws IOException
-     */
-    public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize) throws IOException {
-        this(out, blockSize, false);
-    }
-
-    /**
-     * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
-     * 
-     * @param out The output stream to compress
-     * @throws IOException
-     */
-    public KafkaLZ4BlockOutputStream(OutputStream out) throws IOException {
-        this(out, BLOCKSIZE_64KB);
-    }
-
-    /**
-     * Writes the magic number and frame descriptor to the underlying {@link OutputStream}.
-     * 
-     * @throws IOException
-     */
-    private void writeHeader() throws IOException {
-        Utils.writeUnsignedIntLE(buffer, 0, MAGIC);
-        bufferOffset = 4;
-        buffer[bufferOffset++] = flg.toByte();
-        buffer[bufferOffset++] = bd.toByte();
-        // TODO write uncompressed content size, update flg.validate()
-        // TODO write dictionary id, update flg.validate()
-        // compute checksum on all descriptor fields
-        int hash = (checksum.hash(buffer, 0, bufferOffset, 0) >> 8) & 0xFF;
-        buffer[bufferOffset++] = (byte) hash;
-        // write out frame descriptor
-        out.write(buffer, 0, bufferOffset);
-        bufferOffset = 0;
-    }
-
-    /**
-     * Compresses buffered data, optionally computes an XXHash32 checksum, and writes the result to the underlying
-     * {@link OutputStream}.
-     * 
-     * @throws IOException
-     */
-    private void writeBlock() throws IOException {
-        if (bufferOffset == 0) {
-            return;
-        }
-
-        int compressedLength = compressor.compress(buffer, 0, bufferOffset, compressedBuffer, 0);
-        byte[] bufferToWrite = compressedBuffer;
-        int compressMethod = 0;
-
-        // Store block uncompressed if compressed length is greater (incompressible)
-        if (compressedLength >= bufferOffset) {
-            bufferToWrite = buffer;
-            compressedLength = bufferOffset;
-            compressMethod = LZ4_FRAME_INCOMPRESSIBLE_MASK;
-        }
-
-        // Write content
-        Utils.writeUnsignedIntLE(out, compressedLength | compressMethod);
-        out.write(bufferToWrite, 0, compressedLength);
-
-        // Calculate and write block checksum
-        if (flg.isBlockChecksumSet()) {
-            int hash = checksum.hash(bufferToWrite, 0, compressedLength, 0);
-            Utils.writeUnsignedIntLE(out, hash);
-        }
-        bufferOffset = 0;
-    }
-
-    /**
-     * Similar to the {@link #writeBlock()} method. Writes a 0-length block (without block checksum) to signal the end
-     * of the block stream.
-     * 
-     * @throws IOException
-     */
-    private void writeEndMark() throws IOException {
-        Utils.writeUnsignedIntLE(out, 0);
-        // TODO implement content checksum, update flg.validate()
-        finished = true;
-    }
-
-    @Override
-    public void write(int b) throws IOException {
-        ensureNotFinished();
-        if (bufferOffset == maxBlockSize) {
-            writeBlock();
-        }
-        buffer[bufferOffset++] = (byte) b;
-    }
-
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-        net.jpountz.util.Utils.checkRange(b, off, len);
-        ensureNotFinished();
-
-        int bufferRemainingLength = maxBlockSize - bufferOffset;
-        // while b will fill the buffer
-        while (len > bufferRemainingLength) {
-            // fill remaining space in buffer
-            System.arraycopy(b, off, buffer, bufferOffset, bufferRemainingLength);
-            bufferOffset = maxBlockSize;
-            writeBlock();
-            // compute new offset and length
-            off += bufferRemainingLength;
-            len -= bufferRemainingLength;
-            bufferRemainingLength = maxBlockSize;
-        }
-
-        System.arraycopy(b, off, buffer, bufferOffset, len);
-        bufferOffset += len;
-    }
-
-    @Override
-    public void flush() throws IOException {
-        if (!finished) {
-            writeBlock();
-        }
-        if (out != null) {
-            out.flush();
-        }
-    }
-
-    /**
-     * A simple state check to ensure the stream is still open.
-     */
-    private void ensureNotFinished() {
-        if (finished) {
-            throw new IllegalStateException(CLOSED_STREAM);
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (!finished) {
-            writeEndMark();
-            flush();
-            finished = true;
-        }
-        if (out != null) {
-            out.close();
-            out = null;
-        }
-    }
-
-    public static class FLG {
-
-        private static final int VERSION = 1;
-
-        private final int presetDictionary;
-        private final int reserved1;
-        private final int contentChecksum;
-        private final int contentSize;
-        private final int blockChecksum;
-        private final int blockIndependence;
-        private final int version;
-
-        public FLG() {
-            this(false);
-        }
-
-        public FLG(boolean blockChecksum) {
-            this(0, 0, 0, 0, blockChecksum ? 1 : 0, 1, VERSION);
-        }
-
-        private FLG(int presetDictionary,
-                    int reserved1,
-                    int contentChecksum,
-                    int contentSize,
-                    int blockChecksum,
-                    int blockIndependence,
-                    int version) {
-            this.presetDictionary = presetDictionary;
-            this.reserved1 = reserved1;
-            this.contentChecksum = contentChecksum;
-            this.contentSize = contentSize;
-            this.blockChecksum = blockChecksum;
-            this.blockIndependence = blockIndependence;
-            this.version = version;
-            validate();
-        }
-
-        public static FLG fromByte(byte flg) {
-            int presetDictionary = (flg >>> 0) & 1;
-            int reserved1 = (flg >>> 1) & 1;
-            int contentChecksum = (flg >>> 2) & 1;
-            int contentSize = (flg >>> 3) & 1;
-            int blockChecksum = (flg >>> 4) & 1;
-            int blockIndependence = (flg >>> 5) & 1;
-            int version = (flg >>> 6) & 3;
-
-            return new FLG(presetDictionary,
-                           reserved1,
-                           contentChecksum,
-                           contentSize,
-                           blockChecksum,
-                           blockIndependence,
-                           version);
-        }
-
-        public byte toByte() {
-            return (byte) (((presetDictionary & 1) << 0) | ((reserved1 & 1) << 1) | ((contentChecksum & 1) << 2)
-                    | ((contentSize & 1) << 3) | ((blockChecksum & 1) << 4) | ((blockIndependence & 1) << 5) | ((version & 3) << 6));
-        }
-
-        private void validate() {
-            if (presetDictionary != 0) {
-                throw new RuntimeException("Preset dictionary is unsupported");
-            }
-            if (reserved1 != 0) {
-                throw new RuntimeException("Reserved1 field must be 0");
-            }
-            if (contentChecksum != 0) {
-                throw new RuntimeException("Content checksum is unsupported");
-            }
-            if (contentSize != 0) {
-                throw new RuntimeException("Content size is unsupported");
-            }
-            if (blockIndependence != 1) {
-                throw new RuntimeException("Dependent block stream is unsupported");
-            }
-            if (version != VERSION) {
-                throw new RuntimeException(String.format("Version %d is unsupported", version));
-            }
-        }
-
-        public boolean isPresetDictionarySet() {
-            return presetDictionary == 1;
-        }
-
-        public boolean isContentChecksumSet() {
-            return contentChecksum == 1;
-        }
-
-        public boolean isContentSizeSet() {
-            return contentSize == 1;
-        }
-
-        public boolean isBlockChecksumSet() {
-            return blockChecksum == 1;
-        }
-
-        public boolean isBlockIndependenceSet() {
-            return blockIndependence == 1;
-        }
-
-        public int getVersion() {
-            return version;
-        }
-    }
-
-    public static class BD {
-
-        private final int reserved2;
-        private final int blockSizeValue;
-        private final int reserved3;
-
-        public BD() {
-            this(0, BLOCKSIZE_64KB, 0);
-        }
-
-        public BD(int blockSizeValue) {
-            this(0, blockSizeValue, 0);
-        }
-
-        private BD(int reserved2, int blockSizeValue, int reserved3) {
-            this.reserved2 = reserved2;
-            this.blockSizeValue = blockSizeValue;
-            this.reserved3 = reserved3;
-            validate();
-        }
-
-        public static BD fromByte(byte bd) {
-            int reserved2 = (bd >>> 0) & 15;
-            int blockMaximumSize = (bd >>> 4) & 7;
-            int reserved3 = (bd >>> 7) & 1;
-
-            return new BD(reserved2, blockMaximumSize, reserved3);
-        }
-
-        private void validate() {
-            if (reserved2 != 0) {
-                throw new RuntimeException("Reserved2 field must be 0");
-            }
-            if (blockSizeValue < 4 || blockSizeValue > 7) {
-                throw new RuntimeException("Block size value must be between 4 and 7");
-            }
-            if (reserved3 != 0) {
-                throw new RuntimeException("Reserved3 field must be 0");
-            }
-        }
-
-        // 2^(2n+8)
-        public int getBlockMaximumSize() {
-            return 1 << ((2 * blockSizeValue) + 8);
-        }
-
-        public byte toByte() {
-            return (byte) (((reserved2 & 15) << 0) | ((blockSizeValue & 7) << 4) | ((reserved3 & 1) << 7));
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/LogEntry.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/LogEntry.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/LogEntry.java
deleted file mode 100644
index b5d7b2d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/LogEntry.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.kafka_backport.common.record;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * An offset and record pair
- */
-public final class LogEntry {
-
-    private final long offset;
-    private final Record record;
-
-    public LogEntry(long offset, Record record) {
-        this.offset = offset;
-        this.record = record;
-    }
-
-    public long offset() {
-        return this.offset;
-    }
-
-    public Record record() {
-        return this.record;
-    }
-
-    @Override
-    public String toString() {
-        return "LogEntry(" + offset + ", " + record + ")";
-    }
-    
-    public int size() {
-        return record.size() + Records.LOG_OVERHEAD;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/MemoryRecords.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/MemoryRecords.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/MemoryRecords.java
deleted file mode 100644
index 15e98ed..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/MemoryRecords.java
+++ /dev/null
@@ -1,280 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.record;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-import org.apache.flink.kafka_backport.common.utils.AbstractIterator;
-
-import java.io.DataInputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.GatheringByteChannel;
-import java.util.Iterator;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A {@link Records} implementation backed by a ByteBuffer.
- */
-public class MemoryRecords implements Records {
-
-    private final Compressor compressor;
-    private final int capacity;
-    private final int sizeLimit;
-    private ByteBuffer buffer;
-    private boolean writable;
-
-    // Construct a writable memory records
-    private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int sizeLimit) {
-        this.writable = writable;
-        this.capacity = buffer.capacity();
-        this.sizeLimit = sizeLimit;
-        if (this.writable) {
-            this.buffer = null;
-            this.compressor = new Compressor(buffer, type);
-        } else {
-            this.buffer = buffer;
-            this.compressor = null;
-        }
-    }
-
-    public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int capacity) {
-        return new MemoryRecords(buffer, type, true, capacity);
-    }
-
-    public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) {
-        return emptyRecords(buffer, type, buffer.capacity());
-    }
-
-    public static MemoryRecords readableRecords(ByteBuffer buffer) {
-        return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity());
-    }
-
-    /**
-     * Append the given record and offset to the buffer
-     */
-    public void append(long offset, Record record) {
-        if (!writable)
-            throw new IllegalStateException("Memory records is not writable");
-
-        int size = record.size();
-        compressor.putLong(offset);
-        compressor.putInt(size);
-        compressor.put(record.buffer());
-        compressor.recordWritten(size + LOG_OVERHEAD);
-        record.buffer().rewind();
-    }
-
-    /**
-     * Append a new record and offset to the buffer
-     */
-    public void append(long offset, byte[] key, byte[] value) {
-        if (!writable)
-            throw new IllegalStateException("Memory records is not writable");
-
-        int size = Record.recordSize(key, value);
-        compressor.putLong(offset);
-        compressor.putInt(size);
-        compressor.putRecord(key, value);
-        compressor.recordWritten(size + LOG_OVERHEAD);
-    }
-
-    /**
-     * Check if we have room for a new record containing the given key/value pair
-     * 
-     * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
-     * accurate if compression is really used. When this happens, the following append may cause dynamic buffer
-     * re-allocation in the underlying byte buffer stream.
-     * 
-     * Also note that besides the records' capacity, there is also a size limit for the batch. This size limit may be
-     * smaller than the capacity (e.g. when appending a single message whose size is larger than the batch size, the
-     * capacity will be the message size, but the size limit will still be the batch size), and when the records' size
-     * has exceed this limit we also mark this record as full.
-     */
-    public boolean hasRoomFor(byte[] key, byte[] value) {
-        return this.writable && this.capacity >= this.compressor.estimatedBytesWritten() + LOG_OVERHEAD +
-                                                 Record.recordSize(key, value) &&
-               this.sizeLimit >= this.compressor.estimatedBytesWritten();
-    }
-
-    public boolean isFull() {
-        return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten() ||
-               this.sizeLimit <= this.compressor.estimatedBytesWritten();
-    }
-
-    /**
-     * Close this batch for no more appends
-     */
-    public void close() {
-        if (writable) {
-            compressor.close();
-            writable = false;
-            buffer = compressor.buffer();
-        }
-    }
-
-    /** Write the records in this set to the given channel */
-    public int writeTo(GatheringByteChannel channel) throws IOException {
-        return channel.write(buffer);
-    }
-
-    /**
-     * The size of this record set
-     */
-    public int sizeInBytes() {
-        return compressor.buffer().position();
-    }
-
-    /**
-     * The compression rate of this record set
-     */
-    public double compressionRate() {
-        if (compressor == null)
-            return 1.0;
-        else
-            return compressor.compressionRate();
-    }
-
-    /**
-     * Return the capacity of the buffer
-     */
-    public int capacity() {
-        return this.capacity;
-    }
-
-    /**
-     * Get the byte buffer that backs this records instance
-     */
-    public ByteBuffer buffer() {
-        return buffer.duplicate();
-    }
-
-    /**
-     * Return a flipped duplicate of the closed buffer to reading records
-     */
-    public ByteBuffer flip() {
-        if (writable)
-            throw new IllegalStateException("The memory records need to be closed for write before rewinding for read");
-
-        return (ByteBuffer) buffer.flip();
-    }
-
-    @Override
-    public Iterator<LogEntry> iterator() {
-        ByteBuffer copy = this.buffer.duplicate();
-        return new RecordsIterator(copy, CompressionType.NONE, false);
-    }
-    
-    @Override
-    public String toString() {
-        Iterator<LogEntry> iter = iterator();
-        StringBuilder builder = new StringBuilder();
-        builder.append('[');
-        while (iter.hasNext()) {
-            LogEntry entry = iter.next();
-            builder.append('(');
-            builder.append("offset=");
-            builder.append(entry.offset());
-            builder.append(",");
-            builder.append("record=");
-            builder.append(entry.record());
-            builder.append(")");
-        }
-        builder.append(']');
-        return builder.toString();
-    }
-
-    public static class RecordsIterator extends AbstractIterator<LogEntry> {
-        private final ByteBuffer buffer;
-        private final DataInputStream stream;
-        private final CompressionType type;
-        private final boolean shallow;
-        private RecordsIterator innerIter;
-
-        public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) {
-            this.type = type;
-            this.buffer = buffer;
-            this.shallow = shallow;
-            this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type);
-        }
-
-        /*
-         * Read the next record from the buffer.
-         * 
-         * Note that in the compressed message set, each message value size is set as the size of the un-compressed
-         * version of the message value, so when we do de-compression allocating an array of the specified size for
-         * reading compressed value data is sufficient.
-         */
-        @Override
-        protected LogEntry makeNext() {
-            if (innerDone()) {
-                try {
-                    // read the offset
-                    long offset = stream.readLong();
-                    // read record size
-                    int size = stream.readInt();
-                    if (size < 0)
-                        throw new IllegalStateException("Record with size " + size);
-                    // read the record, if compression is used we cannot depend on size
-                    // and hence has to do extra copy
-                    ByteBuffer rec;
-                    if (type == CompressionType.NONE) {
-                        rec = buffer.slice();
-                        int newPos = buffer.position() + size;
-                        if (newPos > buffer.limit())
-                            return allDone();
-                        buffer.position(newPos);
-                        rec.limit(size);
-                    } else {
-                        byte[] recordBuffer = new byte[size];
-                        stream.readFully(recordBuffer, 0, size);
-                        rec = ByteBuffer.wrap(recordBuffer);
-                    }
-                    LogEntry entry = new LogEntry(offset, new Record(rec));
-
-                    // decide whether to go shallow or deep iteration if it is compressed
-                    CompressionType compression = entry.record().compressionType();
-                    if (compression == CompressionType.NONE || shallow) {
-                        return entry;
-                    } else {
-                        // init the inner iterator with the value payload of the message,
-                        // which will de-compress the payload to a set of messages;
-                        // since we assume nested compression is not allowed, the deep iterator
-                        // would not try to further decompress underlying messages
-                        ByteBuffer value = entry.record().value();
-                        innerIter = new RecordsIterator(value, compression, true);
-                        return innerIter.next();
-                    }
-                } catch (EOFException e) {
-                    return allDone();
-                } catch (IOException e) {
-                    throw new KafkaException(e);
-                }
-            } else {
-                return innerIter.next();
-            }
-        }
-
-        private boolean innerDone() {
-            return innerIter == null || !innerIter.hasNext();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Record.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Record.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Record.java
deleted file mode 100644
index f71900c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Record.java
+++ /dev/null
@@ -1,352 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.record;
-
-import org.apache.flink.kafka_backport.common.utils.Crc32;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A record: a serialized key and value along with the associated CRC and other fields
- */
-public final class Record {
-
-    /**
-     * The current offset and size for all the fixed-length fields
-     */
-    public static final int CRC_OFFSET = 0;
-    public static final int CRC_LENGTH = 4;
-    public static final int MAGIC_OFFSET = CRC_OFFSET + CRC_LENGTH;
-    public static final int MAGIC_LENGTH = 1;
-    public static final int ATTRIBUTES_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
-    public static final int ATTRIBUTE_LENGTH = 1;
-    public static final int KEY_SIZE_OFFSET = ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH;
-    public static final int KEY_SIZE_LENGTH = 4;
-    public static final int KEY_OFFSET = KEY_SIZE_OFFSET + KEY_SIZE_LENGTH;
-    public static final int VALUE_SIZE_LENGTH = 4;
-
-    /**
-     * The size for the record header
-     */
-    public static final int HEADER_SIZE = CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH;
-
-    /**
-     * The amount of overhead bytes in a record
-     */
-    public static final int RECORD_OVERHEAD = HEADER_SIZE + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
-
-    /**
-     * The current "magic" value
-     */
-    public static final byte CURRENT_MAGIC_VALUE = 0;
-
-    /**
-     * Specifies the mask for the compression code. 3 bits to hold the compression codec. 0 is reserved to indicate no
-     * compression
-     */
-    public static final int COMPRESSION_CODEC_MASK = 0x07;
-
-    /**
-     * Compression code for uncompressed records
-     */
-    public static final int NO_COMPRESSION = 0;
-
-    private final ByteBuffer buffer;
-
-    public Record(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    /**
-     * A constructor to create a LogRecord. If the record's compression type is not none, then
-     * its value payload should be already compressed with the specified type; the constructor
-     * would always write the value payload as is and will not do the compression itself.
-     * 
-     * @param key The key of the record (null, if none)
-     * @param value The record value
-     * @param type The compression type used on the contents of the record (if any)
-     * @param valueOffset The offset into the payload array used to extract payload
-     * @param valueSize The size of the payload to use
-     */
-    public Record(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length,
-                value == null ? 0 : valueSize >= 0 ? valueSize : value.length - valueOffset)));
-        write(this.buffer, key, value, type, valueOffset, valueSize);
-        this.buffer.rewind();
-    }
-
-    public Record(byte[] key, byte[] value, CompressionType type) {
-        this(key, value, type, 0, -1);
-    }
-
-    public Record(byte[] value, CompressionType type) {
-        this(null, value, type);
-    }
-
-    public Record(byte[] key, byte[] value) {
-        this(key, value, CompressionType.NONE);
-    }
-
-    public Record(byte[] value) {
-        this(null, value, CompressionType.NONE);
-    }
-
-    // Write a record to the buffer, if the record's compression type is none, then
-    // its value payload should be already compressed with the specified type
-    public static void write(ByteBuffer buffer, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        // construct the compressor with compression type none since this function will not do any
-        //compression according to the input type, it will just write the record's payload as is
-        Compressor compressor = new Compressor(buffer, CompressionType.NONE, buffer.capacity());
-        compressor.putRecord(key, value, type, valueOffset, valueSize);
-    }
-
-    public static void write(Compressor compressor, long crc, byte attributes, byte[] key, byte[] value, int valueOffset, int valueSize) {
-        // write crc
-        compressor.putInt((int) (crc & 0xffffffffL));
-        // write magic value
-        compressor.putByte(CURRENT_MAGIC_VALUE);
-        // write attributes
-        compressor.putByte(attributes);
-        // write the key
-        if (key == null) {
-            compressor.putInt(-1);
-        } else {
-            compressor.putInt(key.length);
-            compressor.put(key, 0, key.length);
-        }
-        // write the value
-        if (value == null) {
-            compressor.putInt(-1);
-        } else {
-            int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
-            compressor.putInt(size);
-            compressor.put(value, valueOffset, size);
-        }
-    }
-
-    public static int recordSize(byte[] key, byte[] value) {
-        return recordSize(key == null ? 0 : key.length, value == null ? 0 : value.length);
-    }
-
-    public static int recordSize(int keySize, int valueSize) {
-        return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
-    }
-
-    public ByteBuffer buffer() {
-        return this.buffer;
-    }
-
-    public static byte computeAttributes(CompressionType type) {
-        byte attributes = 0;
-        if (type.id > 0)
-            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
-        return attributes;
-    }
-
-    /**
-     * Compute the checksum of the record from the record contents
-     */
-    public static long computeChecksum(ByteBuffer buffer, int position, int size) {
-        Crc32 crc = new Crc32();
-        crc.update(buffer.array(), buffer.arrayOffset() + position, size);
-        return crc.getValue();
-    }
-
-    /**
-     * Compute the checksum of the record from the attributes, key and value payloads
-     */
-    public static long computeChecksum(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        Crc32 crc = new Crc32();
-        crc.update(CURRENT_MAGIC_VALUE);
-        byte attributes = 0;
-        if (type.id > 0)
-            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
-        crc.update(attributes);
-        // update for the key
-        if (key == null) {
-            crc.updateInt(-1);
-        } else {
-            crc.updateInt(key.length);
-            crc.update(key, 0, key.length);
-        }
-        // update for the value
-        if (value == null) {
-            crc.updateInt(-1);
-        } else {
-            int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
-            crc.updateInt(size);
-            crc.update(value, valueOffset, size);
-        }
-        return crc.getValue();
-    }
-
-
-    /**
-     * Compute the checksum of the record from the record contents
-     */
-    public long computeChecksum() {
-        return computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
-    }
-
-    /**
-     * Retrieve the previously computed CRC for this record
-     */
-    public long checksum() {
-        return Utils.readUnsignedInt(buffer, CRC_OFFSET);
-    }
-
-    /**
-     * Returns true if the crc stored with the record matches the crc computed off the record contents
-     */
-    public boolean isValid() {
-        return checksum() == computeChecksum();
-    }
-
-    /**
-     * Throw an InvalidRecordException if isValid is false for this record
-     */
-    public void ensureValid() {
-        if (!isValid())
-            throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
-                                             + ", computed crc = "
-                                             + computeChecksum()
-                                             + ")");
-    }
-
-    /**
-     * The complete serialized size of this record in bytes (including crc, header attributes, etc)
-     */
-    public int size() {
-        return buffer.limit();
-    }
-
-    /**
-     * The length of the key in bytes
-     */
-    public int keySize() {
-        return buffer.getInt(KEY_SIZE_OFFSET);
-    }
-
-    /**
-     * Does the record have a key?
-     */
-    public boolean hasKey() {
-        return keySize() >= 0;
-    }
-
-    /**
-     * The position where the value size is stored
-     */
-    private int valueSizeOffset() {
-        return KEY_OFFSET + Math.max(0, keySize());
-    }
-
-    /**
-     * The length of the value in bytes
-     */
-    public int valueSize() {
-        return buffer.getInt(valueSizeOffset());
-    }
-
-    /**
-     * The magic version of this record
-     */
-    public byte magic() {
-        return buffer.get(MAGIC_OFFSET);
-    }
-
-    /**
-     * The attributes stored with this record
-     */
-    public byte attributes() {
-        return buffer.get(ATTRIBUTES_OFFSET);
-    }
-
-    /**
-     * The compression type used with this record
-     */
-    public CompressionType compressionType() {
-        return CompressionType.forId(buffer.get(ATTRIBUTES_OFFSET) & COMPRESSION_CODEC_MASK);
-    }
-
-    /**
-     * A ByteBuffer containing the value of this record
-     */
-    public ByteBuffer value() {
-        return sliceDelimited(valueSizeOffset());
-    }
-
-    /**
-     * A ByteBuffer containing the message key
-     */
-    public ByteBuffer key() {
-        return sliceDelimited(KEY_SIZE_OFFSET);
-    }
-
-    /**
-     * Read a size-delimited byte buffer starting at the given offset
-     */
-    private ByteBuffer sliceDelimited(int start) {
-        int size = buffer.getInt(start);
-        if (size < 0) {
-            return null;
-        } else {
-            ByteBuffer b = buffer.duplicate();
-            b.position(start + 4);
-            b = b.slice();
-            b.limit(size);
-            b.rewind();
-            return b;
-        }
-    }
-
-    public String toString() {
-        return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, key = %d bytes, value = %d bytes)",
-                magic(),
-                attributes(),
-                compressionType(),
-                checksum(),
-                key() == null ? 0 : key().limit(),
-                value() == null ? 0 : value().limit());
-    }
-
-    public boolean equals(Object other) {
-        if (this == other)
-            return true;
-        if (other == null)
-            return false;
-        if (!other.getClass().equals(Record.class))
-            return false;
-        Record record = (Record) other;
-        return this.buffer.equals(record.buffer);
-    }
-
-    public int hashCode() {
-        return buffer.hashCode();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Records.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Records.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Records.java
deleted file mode 100644
index 433748b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Records.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.kafka_backport.common.record;
-
-import java.io.IOException;
-import java.nio.channels.GatheringByteChannel;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A binary format which consists of a 4 byte size, an 8 byte offset, and the record bytes. See {@link MemoryRecords}
- * for the in-memory representation.
- */
-public interface Records extends Iterable<LogEntry> {
-
-    int SIZE_LENGTH = 4;
-    int OFFSET_LENGTH = 8;
-    int LOG_OVERHEAD = SIZE_LENGTH + OFFSET_LENGTH;
-
-    /**
-     * Write these records to the given channel
-     * @param channel The channel to write to
-     * @return The number of bytes written
-     * @throws IOException If the write fails.
-     */
-    public int writeTo(GatheringByteChannel channel) throws IOException;
-
-    /**
-     * The size of these records in bytes
-     */
-    public int sizeInBytes();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequest.java
deleted file mode 100644
index e888d1e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequest.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public abstract class AbstractRequest extends AbstractRequestResponse {
-
-    public AbstractRequest(Struct struct) {
-        super(struct);
-    }
-
-    /**
-     * Get an error response for a request for a given api version
-     */
-    public abstract AbstractRequestResponse getErrorResponse(int versionId, Throwable e);
-
-    /**
-     * Factory method for getting a request object based on ApiKey ID and a buffer
-     */
-    public static AbstractRequest getRequest(int requestId, int versionId, ByteBuffer buffer) {
-        switch (ApiKeys.forId(requestId)) {
-            case PRODUCE:
-                return ProduceRequest.parse(buffer, versionId);
-            case FETCH:
-                return FetchRequest.parse(buffer, versionId);
-            case LIST_OFFSETS:
-                return ListOffsetRequest.parse(buffer, versionId);
-            case METADATA:
-                return MetadataRequest.parse(buffer, versionId);
-            case OFFSET_COMMIT:
-                return OffsetCommitRequest.parse(buffer, versionId);
-            case OFFSET_FETCH:
-                return OffsetFetchRequest.parse(buffer, versionId);
-            case CONSUMER_METADATA:
-                return ConsumerMetadataRequest.parse(buffer, versionId);
-            case JOIN_GROUP:
-                return JoinGroupRequest.parse(buffer, versionId);
-            case HEARTBEAT:
-                return HeartbeatRequest.parse(buffer, versionId);
-            default:
-                return null;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequestResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequestResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequestResponse.java
deleted file mode 100644
index 002e8db..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequestResponse.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public abstract class AbstractRequestResponse {
-    protected final Struct struct;
-
-
-    public AbstractRequestResponse(Struct struct) {
-        this.struct = struct;
-    }
-
-    public Struct toStruct() {
-        return struct;
-    }
-
-    /**
-     * Get the serialized size of this object
-     */
-    public int sizeOf() {
-        return struct.sizeOf();
-    }
-
-    /**
-     * Write this object to a buffer
-     */
-    public void writeTo(ByteBuffer buffer) {
-        struct.writeTo(buffer);
-    }
-
-    @Override
-    public String toString() {
-        return struct.toString();
-    }
-
-    @Override
-    public int hashCode() {
-        return struct.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        AbstractRequestResponse other = (AbstractRequestResponse) obj;
-        return struct.equals(other.struct);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataRequest.java
deleted file mode 100644
index e4f5e90..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataRequest.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.Node;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ConsumerMetadataRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-
-    private final String groupId;
-
-    public ConsumerMetadataRequest(String groupId) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        this.groupId = groupId;
-    }
-
-    public ConsumerMetadataRequest(Struct struct) {
-        super(struct);
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        switch (versionId) {
-            case 0:
-                return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode());
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CONSUMER_METADATA.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public static ConsumerMetadataRequest parse(ByteBuffer buffer, int versionId) {
-        return new ConsumerMetadataRequest(ProtoUtils.parseRequest(ApiKeys.CONSUMER_METADATA.id, versionId, buffer));
-    }
-
-    public static ConsumerMetadataRequest parse(ByteBuffer buffer) {
-        return new ConsumerMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataResponse.java
deleted file mode 100644
index 3688eda..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataResponse.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.Node;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ConsumerMetadataResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id);
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-    private static final String COORDINATOR_KEY_NAME = "coordinator";
-
-    // coordinator level field names
-    private static final String NODE_ID_KEY_NAME = "node_id";
-    private static final String HOST_KEY_NAME = "host";
-    private static final String PORT_KEY_NAME = "port";
-
-    private final short errorCode;
-    private final Node node;
-
-    public ConsumerMetadataResponse(short errorCode, Node node) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(ERROR_CODE_KEY_NAME, errorCode);
-        Struct coordinator = struct.instance(COORDINATOR_KEY_NAME);
-        coordinator.set(NODE_ID_KEY_NAME, node.id());
-        coordinator.set(HOST_KEY_NAME, node.host());
-        coordinator.set(PORT_KEY_NAME, node.port());
-        struct.set(COORDINATOR_KEY_NAME, coordinator);
-        this.errorCode = errorCode;
-        this.node = node;
-    }
-
-    public ConsumerMetadataResponse(Struct struct) {
-        super(struct);
-        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
-        Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME);
-        int nodeId = broker.getInt(NODE_ID_KEY_NAME);
-        String host = broker.getString(HOST_KEY_NAME);
-        int port = broker.getInt(PORT_KEY_NAME);
-        node = new Node(nodeId, host, port);
-    }
-
-    public short errorCode() {
-        return errorCode;
-    }
-
-    public Node node() {
-        return node;
-    }
-
-    public static ConsumerMetadataResponse parse(ByteBuffer buffer) {
-        return new ConsumerMetadataResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file


[05/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Selector.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Selector.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Selector.java
deleted file mode 100644
index d229211..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Selector.java
+++ /dev/null
@@ -1,655 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.network;
-
-import org.apache.kafka.copied.common.KafkaException;
-import org.apache.kafka.copied.common.MetricName;
-import org.apache.kafka.copied.common.metrics.Measurable;
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-import org.apache.kafka.copied.common.metrics.Metrics;
-import org.apache.kafka.copied.common.metrics.Sensor;
-import org.apache.kafka.copied.common.metrics.stats.Avg;
-import org.apache.kafka.copied.common.metrics.stats.Count;
-import org.apache.kafka.copied.common.metrics.stats.Max;
-import org.apache.kafka.copied.common.metrics.stats.Rate;
-import org.apache.kafka.copied.common.utils.SystemTime;
-import org.apache.kafka.copied.common.utils.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.net.ConnectException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.nio.channels.CancelledKeyException;
-import java.nio.channels.ClosedChannelException;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
-import java.nio.channels.UnresolvedAddressException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-/**
- * A nioSelector interface for doing non-blocking multi-connection network I/O.
- * <p>
- * This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and
- * responses.
- * <p>
- * A connection can be added to the nioSelector associated with an integer id by doing
- * 
- * <pre>
- * nioSelector.connect(42, new InetSocketAddress(&quot;google.com&quot;, server.port), 64000, 64000);
- * </pre>
- * 
- * The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating
- * the connection. The successful invocation of this method does not mean a valid connection has been established.
- * 
- * Sending requests, receiving responses, processing connection completions, and disconnections on the existing
- * connections are all done using the <code>poll()</code> call.
- * 
- * <pre>
- * List&lt;NetworkRequest&gt; requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
- * nioSelector.poll(TIMEOUT_MS, requestsToSend);
- * </pre>
- * 
- * The nioSelector maintains several lists that are reset by each call to <code>poll()</code> which are available via
- * various getters. These are reset by each call to <code>poll()</code>.
- * 
- * This class is not thread safe!
- */
-public class Selector implements Selectable {
-
-    private static final Logger log = LoggerFactory.getLogger(Selector.class);
-
-    private final java.nio.channels.Selector nioSelector;
-    private final Map<String, SelectionKey> keys;
-    private final List<Send> completedSends;
-    private final List<NetworkReceive> completedReceives;
-    private final List<String> disconnected;
-    private final List<String> connected;
-    private final List<String> failedSends;
-    private final Time time;
-    private final SelectorMetrics sensors;
-    private final String metricGrpPrefix;
-    private final Map<String, String> metricTags;
-    private final Map<String, Long> lruConnections;
-    private final long connectionsMaxIdleNanos;
-    private final int maxReceiveSize;
-    private final boolean metricsPerConnection;
-    private long currentTimeNanos;
-    private long nextIdleCloseCheckTime;
-
-
-    /**
-     * Create a new nioSelector
-     */
-    public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags, boolean metricsPerConnection) {
-        try {
-            this.nioSelector = java.nio.channels.Selector.open();
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-        this.maxReceiveSize = maxReceiveSize;
-        this.connectionsMaxIdleNanos = connectionMaxIdleMs * 1000 * 1000;
-        this.time = time;
-        this.metricGrpPrefix = metricGrpPrefix;
-        this.metricTags = metricTags;
-        this.keys = new HashMap<String, SelectionKey>();
-        this.completedSends = new ArrayList<Send>();
-        this.completedReceives = new ArrayList<NetworkReceive>();
-        this.connected = new ArrayList<String>();
-        this.disconnected = new ArrayList<String>();
-        this.failedSends = new ArrayList<String>();
-        this.sensors = new SelectorMetrics(metrics);
-        // initial capacity and load factor are default, we set them explicitly because we want to set accessOrder = true
-        this.lruConnections = new LinkedHashMap<String, Long>(16, .75F, true);
-        currentTimeNanos = new SystemTime().nanoseconds();
-        nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos;
-        this.metricsPerConnection = metricsPerConnection;
-    }
-
-    public Selector(long connectionMaxIdleMS, Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags) {
-        this(NetworkReceive.UNLIMITED, connectionMaxIdleMS, metrics, time, metricGrpPrefix, metricTags, true);
-    }
-
-    /**
-     * Begin connecting to the given address and add the connection to this nioSelector associated with the given id
-     * number.
-     * <p>
-     * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)}
-     * call. Check {@link #connected()} to see which (if any) connections have completed after a given poll call.
-     * @param id The id for the new connection
-     * @param address The address to connect to
-     * @param sendBufferSize The send buffer for the new connection
-     * @param receiveBufferSize The receive buffer for the new connection
-     * @throws IllegalStateException if there is already a connection for that id
-     * @throws IOException if DNS resolution fails on the hostname or if the broker is down
-     */
-    @Override
-    public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException {
-        if (this.keys.containsKey(id))
-            throw new IllegalStateException("There is already a connection for id " + id);
-
-        SocketChannel channel = SocketChannel.open();
-        channel.configureBlocking(false);
-        Socket socket = channel.socket();
-        socket.setKeepAlive(true);
-        socket.setSendBufferSize(sendBufferSize);
-        socket.setReceiveBufferSize(receiveBufferSize);
-        socket.setTcpNoDelay(true);
-        try {
-            channel.connect(address);
-        } catch (UnresolvedAddressException e) {
-            channel.close();
-            throw new IOException("Can't resolve address: " + address, e);
-        } catch (IOException e) {
-            channel.close();
-            throw e;
-        }
-        SelectionKey key = channel.register(this.nioSelector, SelectionKey.OP_CONNECT);
-        key.attach(new Transmissions(id));
-        this.keys.put(id, key);
-    }
-
-    /**
-     * Register the nioSelector with an existing channel
-     * Use this on server-side, when a connection is accepted by a different thread but processed by the Selector
-     * Note that we are not checking if the connection id is valid - since the connection already exists
-     */
-    public void register(String id, SocketChannel channel) throws ClosedChannelException {
-        SelectionKey key = channel.register(nioSelector, SelectionKey.OP_READ);
-        key.attach(new Transmissions(id));
-        this.keys.put(id, key);
-    }
-
-    /**
-     * Disconnect any connections for the given id (if there are any). The disconnection is asynchronous and will not be
-     * processed until the next {@link #poll(long, List) poll()} call.
-     */
-    @Override
-    public void disconnect(String id) {
-        SelectionKey key = this.keys.get(id);
-        if (key != null)
-            key.cancel();
-    }
-
-    /**
-     * Interrupt the nioSelector if it is blocked waiting to do I/O.
-     */
-    @Override
-    public void wakeup() {
-        this.nioSelector.wakeup();
-    }
-
-    /**
-     * Close this selector and all associated connections
-     */
-    @Override
-    public void close() {
-        List<String> connections = new LinkedList<String>(keys.keySet());
-        for (String id: connections)
-            close(id);
-
-        try {
-            this.nioSelector.close();
-        } catch (IOException e) {
-            log.error("Exception closing nioSelector:", e);
-        }
-    }
-
-    /**
-     * Queue the given request for sending in the subsequent {@poll(long)} calls
-     * @param send The request to send
-     */
-    public void send(Send send) {
-        SelectionKey key = keyForId(send.destination());
-        Transmissions transmissions = transmissions(key);
-        if (transmissions.hasSend())
-            throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress.");
-        transmissions.send = send;
-        try {
-            key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
-        } catch (CancelledKeyException e) {
-            close(transmissions.id);
-            this.failedSends.add(send.destination());
-        }
-    }
-
-    /**
-     * Do whatever I/O can be done on each connection without blocking. This includes completing connections, completing
-     * disconnections, initiating new sends, or making progress on in-progress sends or receives.
-     * 
-     * When this call is completed the user can check for completed sends, receives, connections or disconnects using
-     * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These
-     * lists will be cleared at the beginning of each {@link #poll(long, List)} call and repopulated by the call if any
-     * completed I/O.
-     * 
-     * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely.
-     * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is
-     *         already an in-progress send
-     */
-    @Override
-    public void poll(long timeout) throws IOException {
-        clear();
-
-        /* check ready keys */
-        long startSelect = time.nanoseconds();
-        int readyKeys = select(timeout);
-        long endSelect = time.nanoseconds();
-        currentTimeNanos = endSelect;
-        this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds());
-
-        if (readyKeys > 0) {
-            Set<SelectionKey> keys = this.nioSelector.selectedKeys();
-            Iterator<SelectionKey> iter = keys.iterator();
-            while (iter.hasNext()) {
-                SelectionKey key = iter.next();
-                iter.remove();
-
-                Transmissions transmissions = transmissions(key);
-                SocketChannel channel = channel(key);
-
-                // register all per-connection metrics at once
-                sensors.maybeRegisterConnectionMetrics(transmissions.id);
-                lruConnections.put(transmissions.id, currentTimeNanos);
-
-                try {
-                    /* complete any connections that have finished their handshake */
-                    if (key.isConnectable()) {
-                        channel.finishConnect();
-                        key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ);
-                        this.connected.add(transmissions.id);
-                        this.sensors.connectionCreated.record();
-                        log.debug("Connection {} created", transmissions.id);
-                    }
-
-                    /* read from any connections that have readable data */
-                    if (key.isReadable()) {
-                        if (!transmissions.hasReceive())
-                            transmissions.receive = new NetworkReceive(maxReceiveSize, transmissions.id);
-                        try {
-                            transmissions.receive.readFrom(channel);
-                        } catch (InvalidReceiveException e) {
-                            log.error("Invalid data received from " + transmissions.id + " closing connection", e);
-                            close(transmissions.id);
-                            this.disconnected.add(transmissions.id);
-                            throw e;
-                        }
-                        if (transmissions.receive.complete()) {
-                            transmissions.receive.payload().rewind();
-                            this.completedReceives.add(transmissions.receive);
-                            this.sensors.recordBytesReceived(transmissions.id, transmissions.receive.payload().limit());
-                            transmissions.clearReceive();
-                        }
-                    }
-
-                    /* write to any sockets that have space in their buffer and for which we have data */
-                    if (key.isWritable()) {
-                        transmissions.send.writeTo(channel);
-                        if (transmissions.send.completed()) {
-                            this.completedSends.add(transmissions.send);
-                            this.sensors.recordBytesSent(transmissions.id, transmissions.send.size());
-                            transmissions.clearSend();
-                            key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
-                        }
-                    }
-
-                    /* cancel any defunct sockets */
-                    if (!key.isValid()) {
-                        close(transmissions.id);
-                        this.disconnected.add(transmissions.id);
-                    }
-                } catch (IOException e) {
-                    String desc = socketDescription(channel);
-                    if (e instanceof EOFException || e instanceof ConnectException)
-                        log.debug("Connection {} disconnected", desc);
-                    else
-                        log.warn("Error in I/O with connection to {}", desc, e);
-                    close(transmissions.id);
-                    this.disconnected.add(transmissions.id);
-                }
-            }
-        }
-        long endIo = time.nanoseconds();
-        this.sensors.ioTime.record(endIo - endSelect, time.milliseconds());
-        maybeCloseOldestConnection();
-    }
-
-    private String socketDescription(SocketChannel channel) {
-        Socket socket = channel.socket();
-        if (socket == null)
-            return "[unconnected socket]";
-        else if (socket.getInetAddress() != null)
-            return socket.getInetAddress().toString();
-        else
-            return socket.getLocalAddress().toString();
-    }
-
-    @Override
-    public List<Send> completedSends() {
-        return this.completedSends;
-    }
-
-    @Override
-    public List<NetworkReceive> completedReceives() {
-        return this.completedReceives;
-    }
-
-    @Override
-    public List<String> disconnected() {
-        return this.disconnected;
-    }
-
-    @Override
-    public List<String> connected() {
-        return this.connected;
-    }
-
-    @Override
-    public void mute(String id) {
-        mute(this.keyForId(id));
-    }
-
-    private void mute(SelectionKey key) {
-        key.interestOps(key.interestOps() & ~SelectionKey.OP_READ);
-    }
-
-    @Override
-    public void unmute(String id) {
-        unmute(this.keyForId(id));
-    }
-
-    private void unmute(SelectionKey key) {
-        key.interestOps(key.interestOps() | SelectionKey.OP_READ);
-    }
-
-    @Override
-    public void muteAll() {
-        for (SelectionKey key : this.keys.values())
-            mute(key);
-    }
-
-    @Override
-    public void unmuteAll() {
-        for (SelectionKey key : this.keys.values())
-            unmute(key);
-    }
-
-    private void maybeCloseOldestConnection() {
-        if (currentTimeNanos > nextIdleCloseCheckTime) {
-            if (lruConnections.isEmpty()) {
-                nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos;
-            } else {
-                Map.Entry<String, Long> oldestConnectionEntry = lruConnections.entrySet().iterator().next();
-                Long connectionLastActiveTime = oldestConnectionEntry.getValue();
-                nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos;
-                if (currentTimeNanos > nextIdleCloseCheckTime) {
-                    String connectionId = oldestConnectionEntry.getKey();
-                    if (log.isTraceEnabled())
-                        log.trace("About to close the idle connection from " + connectionId
-                                + " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis");
-
-                    disconnected.add(connectionId);
-                    close(connectionId);
-                }
-            }
-        }
-    }
-
-    /**
-     * Clear the results from the prior poll
-     */
-    private void clear() {
-        this.completedSends.clear();
-        this.completedReceives.clear();
-        this.connected.clear();
-        this.disconnected.clear();
-        this.disconnected.addAll(this.failedSends);
-        this.failedSends.clear();
-    }
-
-    /**
-     * Check for data, waiting up to the given timeout.
-     * 
-     * @param ms Length of time to wait, in milliseconds. If negative, wait indefinitely.
-     * @return The number of keys ready
-     * @throws IOException
-     */
-    private int select(long ms) throws IOException {
-        if (ms == 0L)
-            return this.nioSelector.selectNow();
-        else if (ms < 0L)
-            return this.nioSelector.select();
-        else
-            return this.nioSelector.select(ms);
-    }
-
-    /**
-     * Begin closing this connection
-     */
-    public void close(String id) {
-        SelectionKey key = keyForId(id);
-        lruConnections.remove(id);
-        SocketChannel channel = channel(key);
-        Transmissions trans = transmissions(key);
-        if (trans != null) {
-            this.keys.remove(trans.id);
-            trans.clearReceive();
-            trans.clearSend();
-        }
-        key.attach(null);
-        key.cancel();
-        try {
-            channel.socket().close();
-            channel.close();
-        } catch (IOException e) {
-            log.error("Exception closing connection to node {}:", trans.id, e);
-        }
-        this.sensors.connectionClosed.record();
-    }
-
-    /**
-     * Get the selection key associated with this numeric id
-     */
-    private SelectionKey keyForId(String id) {
-        SelectionKey key = this.keys.get(id);
-        if (key == null)
-            throw new IllegalStateException("Attempt to write to socket for which there is no open connection. Connection id " + id + " existing connections " + keys.keySet().toString());
-        return key;
-    }
-
-    /**
-     * Get the transmissions for the given connection
-     */
-    private Transmissions transmissions(SelectionKey key) {
-        return (Transmissions) key.attachment();
-    }
-
-    /**
-     * Get the socket channel associated with this selection key
-     */
-    private SocketChannel channel(SelectionKey key) {
-        return (SocketChannel) key.channel();
-    }
-
-    /**
-     * The id and in-progress send and receive associated with a connection
-     */
-    private static class Transmissions {
-        public String id;
-        public Send send;
-        public NetworkReceive receive;
-
-        public Transmissions(String id) {
-            this.id = id;
-        }
-
-        public boolean hasSend() {
-            return this.send != null;
-        }
-
-        public void clearSend() {
-            this.send = null;
-        }
-
-        public boolean hasReceive() {
-            return this.receive != null;
-        }
-
-        public void clearReceive() {
-            this.receive = null;
-        }
-    }
-
-    private class SelectorMetrics {
-        private final Metrics metrics;
-        public final Sensor connectionClosed;
-        public final Sensor connectionCreated;
-        public final Sensor bytesTransferred;
-        public final Sensor bytesSent;
-        public final Sensor bytesReceived;
-        public final Sensor selectTime;
-        public final Sensor ioTime;
-
-        public SelectorMetrics(Metrics metrics) {
-            this.metrics = metrics;
-            String metricGrpName = metricGrpPrefix + "-metrics";
-            StringBuilder tagsSuffix = new StringBuilder();
-
-            for (Map.Entry<String, String> tag: metricTags.entrySet()) {
-                tagsSuffix.append(tag.getKey());
-                tagsSuffix.append("-");
-                tagsSuffix.append(tag.getValue());
-            }
-
-            this.connectionClosed = this.metrics.sensor("connections-closed:" + tagsSuffix.toString());
-            MetricName metricName = new MetricName("connection-close-rate", metricGrpName, "Connections closed per second in the window.", metricTags);
-            this.connectionClosed.add(metricName, new Rate());
-
-            this.connectionCreated = this.metrics.sensor("connections-created:" + tagsSuffix.toString());
-            metricName = new MetricName("connection-creation-rate", metricGrpName, "New connections established per second in the window.", metricTags);
-            this.connectionCreated.add(metricName, new Rate());
-
-            this.bytesTransferred = this.metrics.sensor("bytes-sent-received:" + tagsSuffix.toString());
-            metricName = new MetricName("network-io-rate", metricGrpName, "The average number of network operations (reads or writes) on all connections per second.", metricTags);
-            bytesTransferred.add(metricName, new Rate(new Count()));
-
-            this.bytesSent = this.metrics.sensor("bytes-sent:" + tagsSuffix.toString(), bytesTransferred);
-            metricName = new MetricName("outgoing-byte-rate", metricGrpName, "The average number of outgoing bytes sent per second to all servers.", metricTags);
-            this.bytesSent.add(metricName, new Rate());
-            metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags);
-            this.bytesSent.add(metricName, new Rate(new Count()));
-            metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", metricTags);
-            this.bytesSent.add(metricName, new Avg());
-            metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", metricTags);
-            this.bytesSent.add(metricName, new Max());
-
-            this.bytesReceived = this.metrics.sensor("bytes-received:" + tagsSuffix.toString(), bytesTransferred);
-            metricName = new MetricName("incoming-byte-rate", metricGrpName, "Bytes/second read off all sockets", metricTags);
-            this.bytesReceived.add(metricName, new Rate());
-            metricName = new MetricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags);
-            this.bytesReceived.add(metricName, new Rate(new Count()));
-
-            this.selectTime = this.metrics.sensor("select-time:" + tagsSuffix.toString());
-            metricName = new MetricName("select-rate", metricGrpName, "Number of times the I/O layer checked for new I/O to perform per second", metricTags);
-            this.selectTime.add(metricName, new Rate(new Count()));
-            metricName = new MetricName("io-wait-time-ns-avg", metricGrpName, "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", metricTags);
-            this.selectTime.add(metricName, new Avg());
-            metricName = new MetricName("io-wait-ratio", metricGrpName, "The fraction of time the I/O thread spent waiting.", metricTags);
-            this.selectTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
-
-            this.ioTime = this.metrics.sensor("io-time:" + tagsSuffix.toString());
-            metricName = new MetricName("io-time-ns-avg", metricGrpName, "The average length of time for I/O per select call in nanoseconds.", metricTags);
-            this.ioTime.add(metricName, new Avg());
-            metricName = new MetricName("io-ratio", metricGrpName, "The fraction of time the I/O thread spent doing I/O", metricTags);
-            this.ioTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
-
-            metricName = new MetricName("connection-count", metricGrpName, "The current number of active connections.", metricTags);
-            this.metrics.addMetric(metricName, new Measurable() {
-                public double measure(MetricConfig config, long now) {
-                    return keys.size();
-                }
-            });
-        }
-
-        public void maybeRegisterConnectionMetrics(String connectionId) {
-            if (!connectionId.isEmpty() && metricsPerConnection) {
-                // if one sensor of the metrics has been registered for the connection,
-                // then all other sensors should have been registered; and vice versa
-                String nodeRequestName = "node-" + connectionId + ".bytes-sent";
-                Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
-                if (nodeRequest == null) {
-                    String metricGrpName = metricGrpPrefix + "-node-metrics";
-
-                    Map<String, String> tags = new LinkedHashMap<String, String>(metricTags);
-                    tags.put("node-id", "node-" + connectionId);
-
-                    nodeRequest = this.metrics.sensor(nodeRequestName);
-                    MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags);
-                    nodeRequest.add(metricName, new Rate());
-                    metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", tags);
-                    nodeRequest.add(metricName, new Rate(new Count()));
-                    metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", tags);
-                    nodeRequest.add(metricName, new Avg());
-                    metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", tags);
-                    nodeRequest.add(metricName, new Max());
-
-                    String nodeResponseName = "node-" + connectionId + ".bytes-received";
-                    Sensor nodeResponse = this.metrics.sensor(nodeResponseName);
-                    metricName = new MetricName("incoming-byte-rate", metricGrpName, tags);
-                    nodeResponse.add(metricName, new Rate());
-                    metricName = new MetricName("response-rate", metricGrpName, "The average number of responses received per second.", tags);
-                    nodeResponse.add(metricName, new Rate(new Count()));
-
-                    String nodeTimeName = "node-" + connectionId + ".latency";
-                    Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName);
-                    metricName = new MetricName("request-latency-avg", metricGrpName, tags);
-                    nodeRequestTime.add(metricName, new Avg());
-                    metricName = new MetricName("request-latency-max", metricGrpName, tags);
-                    nodeRequestTime.add(metricName, new Max());
-                }
-            }
-        }
-
-        public void recordBytesSent(String connectionId, long bytes) {
-            long now = time.milliseconds();
-            this.bytesSent.record(bytes, now);
-            if (!connectionId.isEmpty()) {
-                String nodeRequestName = "node-" + connectionId + ".bytes-sent";
-                Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
-                if (nodeRequest != null)
-                    nodeRequest.record(bytes, now);
-            }
-        }
-
-        public void recordBytesReceived(String connection, int bytes) {
-            long now = time.milliseconds();
-            this.bytesReceived.record(bytes, now);
-            if (!connection.isEmpty()) {
-                String nodeRequestName = "node-" + connection + ".bytes-received";
-                Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
-                if (nodeRequest != null)
-                    nodeRequest.record(bytes, now);
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Send.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Send.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Send.java
deleted file mode 100644
index a55b306..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/network/Send.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.kafka.copied.common.network;
-
-import java.io.IOException;
-import java.nio.channels.GatheringByteChannel;
-
-/**
- * This interface models the in-progress sending of data to a destination identified by an integer id.
- */
-public interface Send {
-
-    /**
-     * The numeric id for the destination of this send
-     */
-    public String destination();
-
-    /**
-     * Is this send complete?
-     */
-    public boolean completed();
-
-    /**
-     * Write some as-yet unwritten bytes from this send to the provided channel. It may take multiple calls for the send
-     * to be completely written
-     * @param channel The channel to write to
-     * @return The number of bytes written
-     * @throws IOException If the write fails
-     */
-    public long writeTo(GatheringByteChannel channel) throws IOException;
-
-    /**
-     * Size of the send
-     */
-    public long size();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/ApiKeys.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/ApiKeys.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/ApiKeys.java
deleted file mode 100644
index 2b2f3da..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/ApiKeys.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.kafka.copied.common.protocol;
-
-/**
- * Identifiers for all the Kafka APIs
- */
-public enum ApiKeys {
-    PRODUCE(0, "Produce"),
-    FETCH(1, "Fetch"),
-    LIST_OFFSETS(2, "Offsets"),
-    METADATA(3, "Metadata"),
-    LEADER_AND_ISR(4, "LeaderAndIsr"),
-    STOP_REPLICA(5, "StopReplica"),
-    UPDATE_METADATA_KEY(6, "UpdateMetadata"),
-    CONTROLLED_SHUTDOWN_KEY(7, "ControlledShutdown"),
-    OFFSET_COMMIT(8, "OffsetCommit"),
-    OFFSET_FETCH(9, "OffsetFetch"),
-    CONSUMER_METADATA(10, "ConsumerMetadata"),
-    JOIN_GROUP(11, "JoinGroup"),
-    HEARTBEAT(12, "Heartbeat");
-
-    private static ApiKeys[] codeToType;
-    public static final int MAX_API_KEY;
-
-    static {
-        int maxKey = -1;
-        for (ApiKeys key : ApiKeys.values()) {
-            maxKey = Math.max(maxKey, key.id);
-        }
-        codeToType = new ApiKeys[maxKey + 1];
-        for (ApiKeys key : ApiKeys.values()) {
-            codeToType[key.id] = key;
-        }
-        MAX_API_KEY = maxKey;
-    }
-
-    /** the perminant and immutable id of an API--this can't change ever */
-    public final short id;
-
-    /** an english description of the api--this is for debugging and can change */
-    public final String name;
-
-    private ApiKeys(int id, String name) {
-        this.id = (short) id;
-        this.name = name;
-    }
-
-    public static ApiKeys forId(int id) {
-        return codeToType[id];
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/Errors.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/Errors.java
deleted file mode 100644
index 7703fc0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/Errors.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.protocol;
-
-import org.apache.kafka.copied.common.errors.ApiException;
-import org.apache.kafka.copied.common.errors.ConsumerCoordinatorNotAvailableException;
-import org.apache.kafka.copied.common.errors.CorruptRecordException;
-import org.apache.kafka.copied.common.errors.IllegalGenerationException;
-import org.apache.kafka.copied.common.errors.InvalidRequiredAcksException;
-import org.apache.kafka.copied.common.errors.InvalidTopicException;
-import org.apache.kafka.copied.common.errors.LeaderNotAvailableException;
-import org.apache.kafka.copied.common.errors.NetworkException;
-import org.apache.kafka.copied.common.errors.NotCoordinatorForConsumerException;
-import org.apache.kafka.copied.common.errors.NotEnoughReplicasAfterAppendException;
-import org.apache.kafka.copied.common.errors.NotEnoughReplicasException;
-import org.apache.kafka.copied.common.errors.NotLeaderForPartitionException;
-import org.apache.kafka.copied.common.errors.OffsetLoadInProgressException;
-import org.apache.kafka.copied.common.errors.OffsetMetadataTooLarge;
-import org.apache.kafka.copied.common.errors.OffsetOutOfRangeException;
-import org.apache.kafka.copied.common.errors.RecordBatchTooLargeException;
-import org.apache.kafka.copied.common.errors.RecordTooLargeException;
-import org.apache.kafka.copied.common.errors.TimeoutException;
-import org.apache.kafka.copied.common.errors.UnknownConsumerIdException;
-import org.apache.kafka.copied.common.errors.UnknownServerException;
-import org.apache.kafka.copied.common.errors.UnknownTopicOrPartitionException;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * This class contains all the client-server errors--those errors that must be sent from the server to the client. These
- * are thus part of the protocol. The names can be changed but the error code cannot.
- * 
- * Do not add exceptions that occur only on the client or only on the server here.
- */
-public enum Errors {
-    UNKNOWN(-1, new UnknownServerException("The server experienced an unexpected error when processing the request")),
-    NONE(0, null),
-    OFFSET_OUT_OF_RANGE(1,
-            new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")),
-    CORRUPT_MESSAGE(2,
-            new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")),
-    UNKNOWN_TOPIC_OR_PARTITION(3,
-            new UnknownTopicOrPartitionException("This server does not host this topic-partition.")),
-    // TODO: errorCode 4 for InvalidFetchSize
-    LEADER_NOT_AVAILABLE(5,
-            new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")),
-    NOT_LEADER_FOR_PARTITION(6,
-            new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")),
-    REQUEST_TIMED_OUT(7,
-            new TimeoutException("The request timed out.")),
-    // TODO: errorCode 8 for BrokerNotAvailable
-    REPLICA_NOT_AVAILABLE(9,
-            new ApiException("The replica is not available for the requested topic-partition")),
-    MESSAGE_TOO_LARGE(10,
-            new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")),
-    OFFSET_METADATA_TOO_LARGE(12,
-            new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")),
-    NETWORK_EXCEPTION(13,
-            new NetworkException("The server disconnected before a response was received.")),
-    OFFSET_LOAD_IN_PROGRESS(14,
-            new OffsetLoadInProgressException("The coordinator is loading offsets and can't process requests.")),
-    CONSUMER_COORDINATOR_NOT_AVAILABLE(15,
-            new ConsumerCoordinatorNotAvailableException("The coordinator is not available.")),
-    NOT_COORDINATOR_FOR_CONSUMER(16,
-            new NotCoordinatorForConsumerException("This is not the correct coordinator for this consumer.")),
-    INVALID_TOPIC_EXCEPTION(17,
-            new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")),
-    RECORD_LIST_TOO_LARGE(18,
-            new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")),
-    NOT_ENOUGH_REPLICAS(19,
-            new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")),
-    NOT_ENOUGH_REPLICAS_AFTER_APPEND(20,
-            new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")),
-    INVALID_REQUIRED_ACKS(21,
-            new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")),
-    ILLEGAL_GENERATION(22,
-            new IllegalGenerationException("Specified consumer generation id is not valid.")),
-    INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23,
-            new ApiException("The request partition assignment strategy does not match that of the group.")),
-    UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24,
-            new ApiException("The request partition assignment strategy is unknown to the broker.")),
-    UNKNOWN_CONSUMER_ID(25,
-            new UnknownConsumerIdException("The coordinator is not aware of this consumer.")),
-    INVALID_SESSION_TIMEOUT(26,
-            new ApiException("The session timeout is not within an acceptable range.")),
-    COMMITTING_PARTITIONS_NOT_ASSIGNED(27,
-            new ApiException("Some of the committing partitions are not assigned the committer")),
-    INVALID_COMMIT_OFFSET_SIZE(28,
-            new ApiException("The committing offset data size is not valid"));
-
-    private static Map<Class<?>, Errors> classToError = new HashMap<Class<?>, Errors>();
-    private static Map<Short, Errors> codeToError = new HashMap<Short, Errors>();
-
-    static {
-        for (Errors error : Errors.values()) {
-            codeToError.put(error.code(), error);
-            if (error.exception != null)
-                classToError.put(error.exception.getClass(), error);
-        }
-    }
-
-    private final short code;
-    private final ApiException exception;
-
-    private Errors(int code, ApiException exception) {
-        this.code = (short) code;
-        this.exception = exception;
-    }
-
-    /**
-     * An instance of the exception
-     */
-    public ApiException exception() {
-        return this.exception;
-    }
-
-    /**
-     * The error code for the exception
-     */
-    public short code() {
-        return this.code;
-    }
-
-    /**
-     * Throw the exception corresponding to this error if there is one
-     */
-    public void maybeThrow() {
-        if (exception != null) {
-            throw this.exception;
-        }
-    }
-
-    /**
-     * Throw the exception if there is one
-     */
-    public static Errors forCode(short code) {
-        Errors error = codeToError.get(code);
-        return error == null ? UNKNOWN : error;
-    }
-
-    /**
-     * Return the error instance associated with this exception (or UKNOWN if there is none)
-     */
-    public static Errors forException(Throwable t) {
-        Errors error = classToError.get(t.getClass());
-        return error == null ? UNKNOWN : error;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/ProtoUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/ProtoUtils.java
deleted file mode 100644
index 767964f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/ProtoUtils.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.kafka.copied.common.protocol;
-
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-public class ProtoUtils {
-
-    private static Schema schemaFor(Schema[][] schemas, int apiKey, int version) {
-        if (apiKey < 0 || apiKey > schemas.length)
-            throw new IllegalArgumentException("Invalid api key: " + apiKey);
-        Schema[] versions = schemas[apiKey];
-        if (version < 0 || version > versions.length)
-            throw new IllegalArgumentException("Invalid version for API key " + apiKey + ": " + version);
-        return versions[version];
-    }
-
-    public static short latestVersion(int apiKey) {
-        if (apiKey < 0 || apiKey >= Protocol.CURR_VERSION.length)
-            throw new IllegalArgumentException("Invalid api key: " + apiKey);
-        return Protocol.CURR_VERSION[apiKey];
-    }
-
-    public static Schema requestSchema(int apiKey, int version) {
-        return schemaFor(Protocol.REQUESTS, apiKey, version);
-    }
-
-    public static Schema currentRequestSchema(int apiKey) {
-        return requestSchema(apiKey, latestVersion(apiKey));
-    }
-
-    public static Schema responseSchema(int apiKey, int version) {
-        return schemaFor(Protocol.RESPONSES, apiKey, version);
-    }
-
-    public static Schema currentResponseSchema(int apiKey) {
-        return schemaFor(Protocol.RESPONSES, apiKey, latestVersion(apiKey));
-    }
-
-    public static Struct parseRequest(int apiKey, int version, ByteBuffer buffer) {
-        return (Struct) requestSchema(apiKey, version).read(buffer);
-    }
-
-    public static Struct parseResponse(int apiKey, ByteBuffer buffer) {
-        return (Struct) currentResponseSchema(apiKey).read(buffer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/Protocol.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/Protocol.java
deleted file mode 100644
index dc33ae6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/Protocol.java
+++ /dev/null
@@ -1,470 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.protocol;
-
-import org.apache.kafka.copied.common.protocol.types.ArrayOf;
-import org.apache.kafka.copied.common.protocol.types.Field;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-
-import static org.apache.kafka.copied.common.protocol.types.Type.BYTES;
-import static org.apache.kafka.copied.common.protocol.types.Type.INT16;
-import static org.apache.kafka.copied.common.protocol.types.Type.INT32;
-import static org.apache.kafka.copied.common.protocol.types.Type.INT64;
-import static org.apache.kafka.copied.common.protocol.types.Type.STRING;
-
-public class Protocol {
-
-    public static final Schema REQUEST_HEADER = new Schema(new Field("api_key", INT16, "The id of the request type."),
-                                                           new Field("api_version", INT16, "The version of the API."),
-                                                           new Field("correlation_id",
-                                                                     INT32,
-                                                                     "A user-supplied integer value that will be passed back with the response"),
-                                                           new Field("client_id",
-                                                                     STRING,
-                                                                     "A user specified identifier for the client making the request."));
-
-    public static final Schema RESPONSE_HEADER = new Schema(new Field("correlation_id",
-                                                                      INT32,
-                                                                      "The user-supplied value passed in with the request"));
-
-    /* Metadata api */
-
-    public static final Schema METADATA_REQUEST_V0 = new Schema(new Field("topics",
-                                                                          new ArrayOf(STRING),
-                                                                          "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics."));
-
-    public static final Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."),
-                                                   new Field("host", STRING, "The hostname of the broker."),
-                                                   new Field("port",
-                                                             INT32,
-                                                             "The port on which the broker accepts requests."));
-
-    public static final Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code",
-                                                                            INT16,
-                                                                            "The error code for the partition, if any."),
-                                                                  new Field("partition_id",
-                                                                            INT32,
-                                                                            "The id of the partition."),
-                                                                  new Field("leader",
-                                                                            INT32,
-                                                                            "The id of the broker acting as leader for this partition."),
-                                                                  new Field("replicas",
-                                                                            new ArrayOf(INT32),
-                                                                            "The set of all nodes that host this partition."),
-                                                                  new Field("isr",
-                                                                            new ArrayOf(INT32),
-                                                                            "The set of nodes that are in sync with the leader for this partition."));
-
-    public static final Schema TOPIC_METADATA_V0 = new Schema(new Field("topic_error_code",
-                                                                        INT16,
-                                                                        "The error code for the given topic."),
-                                                              new Field("topic", STRING, "The name of the topic"),
-                                                              new Field("partition_metadata",
-                                                                        new ArrayOf(PARTITION_METADATA_V0),
-                                                                        "Metadata for each partition of the topic."));
-
-    public static final Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers",
-                                                                           new ArrayOf(BROKER),
-                                                                           "Host and port information for all brokers."),
-                                                                 new Field("topic_metadata",
-                                                                           new ArrayOf(TOPIC_METADATA_V0)));
-
-    public static final Schema[] METADATA_REQUEST = new Schema[] {METADATA_REQUEST_V0};
-    public static final Schema[] METADATA_RESPONSE = new Schema[] {METADATA_RESPONSE_V0};
-
-    /* Produce api */
-
-    public static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING),
-                                                                  new Field("data", new ArrayOf(new Schema(new Field("partition", INT32),
-                                                                                                     new Field("record_set", BYTES)))));
-
-    public static final Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks",
-                                                                   INT16,
-                                                                   "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."),
-                                                               new Field("timeout", INT32, "The time to await a response in ms."),
-                                                               new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0)));
-
-    public static final Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                    new ArrayOf(new Schema(new Field("topic", STRING),
-                                                                                           new Field("partition_responses",
-                                                                                                     new ArrayOf(new Schema(new Field("partition",
-                                                                                                                                      INT32),
-                                                                                                                            new Field("error_code",
-                                                                                                                                      INT16),
-                                                                                                                            new Field("base_offset",
-                                                                                                                                      INT64))))))));
-
-    public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0};
-    public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0};
-
-    /* Offset commit api */
-    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                         INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         INT64,
-                                                                                         "Message offset to be committed."),
-                                                                               new Field("metadata",
-                                                                                         STRING,
-                                                                                         "Any associated metadata the client wants to keep."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition",
-                                                                                         INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         INT64,
-                                                                                         "Message offset to be committed."),
-                                                                               new Field("timestamp",
-                                                                                         INT64,
-                                                                                         "Timestamp of the commit"),
-                                                                               new Field("metadata",
-                                                                                         STRING,
-                                                                                         "Any associated metadata the client wants to keep."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V2 = new Schema(new Field("partition",
-                                                                                         INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         INT64,
-                                                                                         "Message offset to be committed."),
-                                                                               new Field("metadata",
-                                                                                         STRING,
-                                                                                         "Any associated metadata the client wants to keep."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
-                                                                                     STRING,
-                                                                                     "Topic to commit."),
-                                                                           new Field("partitions",
-                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0),
-                                                                                     "Partitions to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(new Field("topic",
-                                                                                     STRING,
-                                                                                     "Topic to commit."),
-                                                                           new Field("partitions",
-                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1),
-                                                                                     "Partitions to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V2 = new Schema(new Field("topic",
-                                                                                     STRING,
-                                                                                     "Topic to commit."),
-                                                                           new Field("partitions",
-                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V2),
-                                                                                     "Partitions to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                               STRING,
-                                                                               "The consumer group id."),
-                                                                     new Field("topics",
-                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
-                                                                               "Topics to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id",
-                                                                               STRING,
-                                                                               "The consumer group id."),
-                                                                     new Field("group_generation_id",
-                                                                               INT32,
-                                                                               "The generation of the consumer group."),
-                                                                     new Field("consumer_id",
-                                                                               STRING,
-                                                                               "The consumer id assigned by the group coordinator."),
-                                                                     new Field("topics",
-                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1),
-                                                                               "Topics to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id",
-                                                                               STRING,
-                                                                               "The consumer group id."),
-                                                                     new Field("group_generation_id",
-                                                                               INT32,
-                                                                               "The generation of the consumer group."),
-                                                                     new Field("consumer_id",
-                                                                               STRING,
-                                                                               "The consumer id assigned by the group coordinator."),
-                                                                     new Field("retention_time",
-                                                                               INT64,
-                                                                               "Time period in ms to retain the offset."),
-                                                                     new Field("topics",
-                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2),
-                                                                               "Topics to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                          INT32,
-                                                                                          "Topic partition id."),
-                                                                                new Field("error_code",
-                                                                                          INT16));
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                            new Field("partition_responses",
-                                                                                      new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0)));
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                                new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] OFFSET_COMMIT_REQUEST = new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2};
-
-    /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */
-    public static final Schema OFFSET_COMMIT_RESPONSE_V1 = OFFSET_COMMIT_RESPONSE_V0;
-    public static final Schema OFFSET_COMMIT_RESPONSE_V2 = OFFSET_COMMIT_RESPONSE_V0;
-
-    public static final Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V1, OFFSET_COMMIT_RESPONSE_V2};
-
-    /* Offset fetch api */
-
-    /*
-     * Wire formats of version 0 and 1 are the same, but with different functionality.
-     * Version 0 will read the offsets from ZK;
-     * Version 1 will read the offsets from Kafka.
-     */
-    public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                        INT32,
-                                                                                        "Topic partition id."));
-
-    public static final Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
-                                                                                    STRING,
-                                                                                    "Topic to fetch offset."),
-                                                                          new Field("partitions",
-                                                                                    new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0),
-                                                                                    "Partitions to fetch offsets."));
-
-    public static final Schema OFFSET_FETCH_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                              STRING,
-                                                                              "The consumer group id."),
-                                                                    new Field("topics",
-                                                                              new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0),
-                                                                              "Topics to fetch offsets."));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                         INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         INT64,
-                                                                                         "Last committed message offset."),
-                                                                               new Field("metadata",
-                                                                                         STRING,
-                                                                                         "Any associated metadata the client wants to keep."),
-                                                                               new Field("error_code", INT16));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                           new Field("partition_responses",
-                                                                                     new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0)));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                               new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)));
-
-    public static final Schema OFFSET_FETCH_REQUEST_V1 = OFFSET_FETCH_REQUEST_V0;
-    public static final Schema OFFSET_FETCH_RESPONSE_V1 = OFFSET_FETCH_RESPONSE_V0;
-
-    public static final Schema[] OFFSET_FETCH_REQUEST = new Schema[] {OFFSET_FETCH_REQUEST_V0, OFFSET_FETCH_REQUEST_V1};
-    public static final Schema[] OFFSET_FETCH_RESPONSE = new Schema[] {OFFSET_FETCH_RESPONSE_V0, OFFSET_FETCH_RESPONSE_V1};
-
-    /* List offset api */
-    public static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                       INT32,
-                                                                                       "Topic partition id."),
-                                                                             new Field("timestamp", INT64, "Timestamp."),
-                                                                             new Field("max_num_offsets",
-                                                                                       INT32,
-                                                                                       "Maximum offsets to return."));
-
-    public static final Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
-                                                                                   STRING,
-                                                                                   "Topic to list offset."),
-                                                                         new Field("partitions",
-                                                                                   new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0),
-                                                                                   "Partitions to list offset."));
-
-    public static final Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id",
-                                                                             INT32,
-                                                                             "Broker id of the follower. For normal consumers, use -1."),
-                                                                   new Field("topics",
-                                                                             new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0),
-                                                                             "Topics to list offsets."));
-
-    public static final Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                        INT32,
-                                                                                        "Topic partition id."),
-                                                                              new Field("error_code", INT16),
-                                                                              new Field("offsets",
-                                                                                        new ArrayOf(INT64),
-                                                                                        "A list of offsets."));
-
-    public static final Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                          new Field("partition_responses",
-                                                                                    new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0)));
-
-    public static final Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                              new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] LIST_OFFSET_REQUEST = new Schema[] {LIST_OFFSET_REQUEST_V0};
-    public static final Schema[] LIST_OFFSET_RESPONSE = new Schema[] {LIST_OFFSET_RESPONSE_V0};
-
-    /* Fetch api */
-    public static final Schema FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                 INT32,
-                                                                                 "Topic partition id."),
-                                                                       new Field("fetch_offset",
-                                                                                 INT64,
-                                                                                 "Message offset."),
-                                                                       new Field("max_bytes",
-                                                                                 INT32,
-                                                                                 "Maximum bytes to fetch."));
-
-    public static final Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", STRING, "Topic to fetch."),
-                                                                   new Field("partitions",
-                                                                             new ArrayOf(FETCH_REQUEST_PARTITION_V0),
-                                                                             "Partitions to fetch."));
-
-    public static final Schema FETCH_REQUEST_V0 = new Schema(new Field("replica_id",
-                                                                       INT32,
-                                                                       "Broker id of the follower. For normal consumers, use -1."),
-                                                             new Field("max_wait_time",
-                                                                       INT32,
-                                                                       "Maximum time in ms to wait for the response."),
-                                                             new Field("min_bytes",
-                                                                       INT32,
-                                                                       "Minimum bytes to accumulate in the response."),
-                                                             new Field("topics",
-                                                                       new ArrayOf(FETCH_REQUEST_TOPIC_V0),
-                                                                       "Topics to fetch."));
-
-    public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                  INT32,
-                                                                                  "Topic partition id."),
-                                                                        new Field("error_code", INT16),
-                                                                        new Field("high_watermark",
-                                                                                  INT64,
-                                                                                  "Last committed offset."),
-                                                                        new Field("record_set", BYTES));
-
-    public static final Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                    new Field("partition_responses",
-                                                                              new ArrayOf(FETCH_RESPONSE_PARTITION_V0)));
-
-    public static final Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                        new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0};
-    public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0};
-
-    /* Consumer metadata api */
-    public static final Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                                   STRING,
-                                                                                   "The consumer group id."));
-
-    public static final Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
-                                                                          new Field("coordinator",
-                                                                                    BROKER,
-                                                                                    "Host and port information for the coordinator for a consumer group."));
-
-    public static final Schema[] CONSUMER_METADATA_REQUEST = new Schema[] {CONSUMER_METADATA_REQUEST_V0};
-    public static final Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] {CONSUMER_METADATA_RESPONSE_V0};
-
-    /* Join group api */
-    public static final Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                            STRING,
-                                                                            "The consumer group id."),
-                                                                  new Field("session_timeout",
-                                                                            INT32,
-                                                                            "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."),
-                                                                  new Field("topics",
-                                                                            new ArrayOf(STRING),
-                                                                            "An array of topics to subscribe to."),
-                                                                  new Field("consumer_id",
-                                                                            STRING,
-                                                                            "The assigned consumer id or an empty string for a new consumer."),
-                                                                  new Field("partition_assignment_strategy",
-                                                                            STRING,
-                                                                            "The strategy for the coordinator to assign partitions."));
-
-    public static final Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                         new Field("partitions", new ArrayOf(INT32)));
-    public static final Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
-                                                                   new Field("group_generation_id",
-                                                                             INT32,
-                                                                             "The generation of the consumer group."),
-                                                                   new Field("consumer_id",
-                                                                             STRING,
-                                                                             "The consumer id assigned by the group coordinator."),
-                                                                   new Field("assigned_partitions",
-                                                                             new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] JOIN_GROUP_REQUEST = new Schema[] {JOIN_GROUP_REQUEST_V0};
-    public static final Schema[] JOIN_GROUP_RESPONSE = new Schema[] {JOIN_GROUP_RESPONSE_V0};
-
-    /* Heartbeat api */
-    public static final Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id", STRING, "The consumer group id."),
-                                                                 new Field("group_generation_id",
-                                                                           INT32,
-                                                                           "The generation of the consumer group."),
-                                                                 new Field("consumer_id",
-                                                                           STRING,
-                                                                           "The consumer id assigned by the group coordinator."));
-
-    public static final Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", INT16));
-
-    public static final Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0};
-    public static final Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0};
-
-    /* an array of all requests and responses with all schema versions */
-    public static final Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][];
-    public static final Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][];
-
-    /* the latest version of each api */
-    public static final short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1];
-
-    static {
-        REQUESTS[ApiKeys.PRODUCE.id] = PRODUCE_REQUEST;
-        REQUESTS[ApiKeys.FETCH.id] = FETCH_REQUEST;
-        REQUESTS[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_REQUEST;
-        REQUESTS[ApiKeys.METADATA.id] = METADATA_REQUEST;
-        REQUESTS[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
-        REQUESTS[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
-        REQUESTS[ApiKeys.UPDATE_METADATA_KEY.id] = new Schema[] {};
-        REQUESTS[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = new Schema[] {};
-        REQUESTS[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_REQUEST;
-        REQUESTS[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_REQUEST;
-        REQUESTS[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_REQUEST;
-        REQUESTS[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_REQUEST;
-        REQUESTS[ApiKeys.HEARTBEAT.id] = HEARTBEAT_REQUEST;
-
-        RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
-        RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE;
-        RESPONSES[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_RESPONSE;
-        RESPONSES[ApiKeys.METADATA.id] = METADATA_RESPONSE;
-        RESPONSES[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
-        RESPONSES[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
-        RESPONSES[ApiKeys.UPDATE_METADATA_KEY.id] = new Schema[] {};
-        RESPONSES[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = new Schema[] {};
-        RESPONSES[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_RESPONSE;
-        RESPONSES[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_RESPONSE;
-        RESPONSES[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_RESPONSE;
-        RESPONSES[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_RESPONSE;
-        RESPONSES[ApiKeys.HEARTBEAT.id] = HEARTBEAT_RESPONSE;
-
-        /* set the maximum version of each api */
-        for (ApiKeys api : ApiKeys.values())
-            CURR_VERSION[api.id] = (short) (REQUESTS[api.id].length - 1);
-
-        /* sanity check that we have the same number of request and response versions for each api */
-        for (ApiKeys api : ApiKeys.values())
-            if (REQUESTS[api.id].length != RESPONSES[api.id].length)
-                throw new IllegalStateException(REQUESTS[api.id].length + " request versions for api " + api.name
-                        + " but " + RESPONSES[api.id].length + " response versions.");
-    }
-
-}


[34/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java
deleted file mode 100644
index fe6684d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.kafka.api.KafkaSource;
-import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
-
-public class KafkaConsumerExample {
-
-	private static String host;
-	private static int port;
-	private static String topic;
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setParallelism(4);
-
-		DataStream<String> kafkaStream = env
-				.addSource(new KafkaSource<String>(host + ":" + port, topic, new JavaDefaultStringSchema()));
-		kafkaStream.print();
-
-		env.execute();
-	}
-
-	private static boolean parseParameters(String[] args) {
-		if (args.length == 3) {
-			host = args[0];
-			port = Integer.parseInt(args[1]);
-			topic = args[2];
-			return true;
-		} else {
-			System.err.println("Usage: KafkaConsumerExample <host> <port> <topic>");
-			return false;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java
deleted file mode 100644
index f241d1c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.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.kafka;
-
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kafka.api.KafkaSink;
-import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
-
-@SuppressWarnings("serial")
-public class KafkaProducerExample {
-
-	public static void main(String[] args) throws Exception {
-		
-		if (args.length < 3) {
-			System.err.println("Usage: KafkaProducerExample <host> <port> <topic>");
-			return;
-		}
-
-		final String host = args[0];
-		final int port = Integer.parseInt(args[1]);
-		final String topic = args[2];
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setParallelism(4);
-		
-		env.addSource(new SourceFunction<String>() {
-			
-			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);
-					Thread.sleep(100L);
-				}
-
-				ctx.collect("q");
-			}
-
-			@Override
-			public void cancel() {
-				running = false;
-			}
-
-
-		})
-			.addSink(new KafkaSink<String>(host + ":" + port, topic, new JavaDefaultStringSchema()))
-				.setParallelism(3);
-
-		env.execute();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
index 0965b29..ead24f3 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
@@ -14,180 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.flink.streaming.connectors.kafka.api;
 
-import java.util.Map;
-import java.util.Properties;
 
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.connectors.kafka.api.config.PartitionerWrapper;
-import org.apache.flink.streaming.connectors.kafka.partitioner.SerializableKafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flink.util.NetUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
-
-import kafka.javaapi.producer.Producer;
-import kafka.producer.KeyedMessage;
-import kafka.producer.ProducerConfig;
-import kafka.serializer.DefaultEncoder;
-
 
 /**
  * Sink that emits its inputs to a Kafka topic.
  *
- * @param <IN>
- * 		Type of the sink input
+ * The KafkaSink has been relocated to org.apache.flink.streaming.connectors.KafkaSink.
+ * This class will be removed in future releases of Flink.
  */
-public class KafkaSink<IN> extends RichSinkFunction<IN> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(KafkaSink.class);
-
-	private Producer<IN, byte[]> producer;
-	private Properties userDefinedProperties;
-	private String topicId;
-	private String brokerList;
-	private SerializationSchema<IN, byte[]> schema;
-	private SerializableKafkaPartitioner partitioner;
-	private Class<? extends SerializableKafkaPartitioner> partitionerClass = null;
-
-	/**
-	 * Creates a KafkaSink for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 *			Addresses of the brokers
-	 * @param topicId
-	 * 		ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 		User defined serialization schema.
-	 */
-	public KafkaSink(String brokerList, String topicId,
-			SerializationSchema<IN, byte[]> serializationSchema) {
-		this(brokerList, topicId, new Properties(), serializationSchema);
+@Deprecated
+public class KafkaSink<IN> extends org.apache.flink.streaming.connectors.KafkaSink<IN> {
+	public KafkaSink(String brokerList, String topicId, SerializationSchema<IN, byte[]> serializationSchema) {
+		super(brokerList, topicId, serializationSchema);
 	}
-
-	/**
-	 * Creates a KafkaSink for a given topic with custom Producer configuration.
-	 * If you use this constructor, the broker should be set with the "metadata.broker.list"
-	 * configuration.
-	 *
-	 * @param brokerList
-	 * 		Addresses of the brokers
-	 * @param topicId
-	 * 		ID of the Kafka topic.
-	 * @param producerConfig
-	 * 		Configurations of the Kafka producer
-	 * @param serializationSchema
-	 * 		User defined serialization schema.
-	 */
-	public KafkaSink(String brokerList, String topicId, Properties producerConfig,
-			SerializationSchema<IN, byte[]> serializationSchema) {
-		String[] elements = brokerList.split(",");
-		for(String broker: elements) {
-			NetUtils.ensureCorrectHostnamePort(broker);
-		}
-		Preconditions.checkNotNull(topicId, "TopicID not set");
-
-		this.brokerList = brokerList;
-		this.topicId = topicId;
-		this.schema = serializationSchema;
-		this.partitionerClass = null;
-		this.userDefinedProperties = producerConfig;
-	}
-
-	/**
-	 * Creates a KafkaSink for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 * @param topicId
-	 * 		ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 		User defined serialization schema.
-	 * @param partitioner
-	 * 		User defined partitioner.
-	 */
-	public KafkaSink(String brokerList, String topicId,
-			SerializationSchema<IN, byte[]> serializationSchema, SerializableKafkaPartitioner partitioner) {
-		this(brokerList, topicId, serializationSchema);
-		ClosureCleaner.ensureSerializable(partitioner);
-		this.partitioner = partitioner;
-	}
-
-	public KafkaSink(String brokerList,
-			String topicId,
-			SerializationSchema<IN, byte[]> serializationSchema,
-			Class<? extends SerializableKafkaPartitioner> partitioner) {
-		this(brokerList, topicId, serializationSchema);
-		this.partitionerClass = partitioner;
-	}
-
-	/**
-	 * Initializes the connection to Kafka.
-	 */
-	@Override
-	public void open(Configuration configuration) {
-
-		Properties properties = new Properties();
-
-		properties.put("metadata.broker.list", brokerList);
-		properties.put("request.required.acks", "-1");
-		properties.put("message.send.max.retries", "10");
-
-		properties.put("serializer.class", DefaultEncoder.class.getCanonicalName());
-
-		// this will not be used as the key will not be serialized
-		properties.put("key.serializer.class", DefaultEncoder.class.getCanonicalName());
-
-		for (Map.Entry<Object, Object> propertiesEntry : userDefinedProperties.entrySet()) {
-			properties.put(propertiesEntry.getKey(), propertiesEntry.getValue());
-		}
-
-		if (partitioner != null) {
-			properties.put("partitioner.class", PartitionerWrapper.class.getCanonicalName());
-			// java serialization will do the rest.
-			properties.put(PartitionerWrapper.SERIALIZED_WRAPPER_NAME, partitioner);
-		}
-		if (partitionerClass != null) {
-			properties.put("partitioner.class", partitionerClass);
-		}
-
-		ProducerConfig config = new ProducerConfig(properties);
-
-		try {
-			producer = new Producer<IN, byte[]>(config);
-		} catch (NullPointerException e) {
-			throw new RuntimeException("Cannot connect to Kafka broker " + brokerList, e);
-		}
-	}
-
-	/**
-	 * Called when new data arrives to the sink, and forwards it to Kafka.
-	 *
-	 * @param next
-	 * 		The incoming data
-	 */
-	@Override
-	public void invoke(IN next) {
-		byte[] serialized = schema.serialize(next);
-
-		// Sending message without serializable key.
-		producer.send(new KeyedMessage<IN, byte[]>(topicId, null, next, serialized));
-	}
-
-	@Override
-	public void close() {
-		if (producer != null) {
-			producer.close();
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSource.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSource.java
deleted file mode 100644
index 3bcbfa7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSource.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.api;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import com.google.common.base.Preconditions;
-import kafka.consumer.Consumer;
-import kafka.consumer.ConsumerConfig;
-import kafka.consumer.ConsumerIterator;
-import kafka.consumer.KafkaStream;
-import kafka.javaapi.consumer.ConsumerConnector;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.source.ConnectorSource;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Source that listens to a Kafka topic using the high level Kafka API.
- * 
- * <p><b>IMPORTANT:</b> This source is not participating in the checkpointing procedure
- * and hence gives no form of processing guarantees.
- * Use the {@link org.apache.flink.streaming.connectors.kafka.api.persistent.PersistentKafkaSource}
- * for a fault tolerant source that provides "exactly once" processing guarantees when used with
- * checkpointing enabled.</p>
- *
- * @param <OUT>
- *            Type of the messages on the topic.
- */
-public class KafkaSource<OUT> extends ConnectorSource<OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class);
-	
-	private static final String DEFAULT_GROUP_ID = "flink-group";
-	private static final long ZOOKEEPER_DEFAULT_SYNC_TIME = 200;
-
-	private final String zookeeperAddress;
-	private final String groupId;
-	private final String topicId;
-	private final Properties customProperties;
-	private final long zookeeperSyncTimeMillis;
-	
-	private transient ConsumerConnector consumer;
-	private transient ConsumerIterator<byte[], byte[]> consumerIterator;
-	
-	private volatile boolean isRunning;
-
-	/**
-	 * Creates a KafkaSource that consumes a topic.
-	 *
-	 * @param zookeeperAddress
-	 *            Address of the Zookeeper host (with port number).
-	 * @param topicId
-	 *            ID of the Kafka topic.
-	 * @param groupId
-	 * 			   ID of the consumer group.
-	 * @param deserializationSchema
-	 *            User defined deserialization schema.
-	 * @param zookeeperSyncTimeMillis
-	 *            Synchronization time with zookeeper.
-	 */
-	public KafkaSource(String zookeeperAddress,
-		String topicId,
-		String groupId,
-		DeserializationSchema<OUT> deserializationSchema,
-		long zookeeperSyncTimeMillis) {
-		this(zookeeperAddress, topicId, groupId, deserializationSchema, zookeeperSyncTimeMillis, null);
-	}
-	/**
-	 * Creates a KafkaSource that consumes a topic.
-	 *
-	 * @param zookeeperAddress
-	 *            Address of the Zookeeper host (with port number).
-	 * @param topicId
-	 *            ID of the Kafka topic.
-	 * @param groupId
-	 * 			   ID of the consumer group.
-	 * @param deserializationSchema
-	 *            User defined deserialization schema.
-	 * @param zookeeperSyncTimeMillis
-	 *            Synchronization time with zookeeper.
-	 * @param customProperties
-	 * 			  Custom properties for Kafka
-	 */
-	public KafkaSource(String zookeeperAddress,
-						String topicId, String groupId,
-						DeserializationSchema<OUT> deserializationSchema,
-						long zookeeperSyncTimeMillis, Properties customProperties)
-	{
-		super(deserializationSchema);
-		
-		Preconditions.checkNotNull(zookeeperAddress, "ZK address is null");
-		Preconditions.checkNotNull(topicId, "Topic ID is null");
-		Preconditions.checkNotNull(deserializationSchema, "deserializationSchema is null");
-		Preconditions.checkArgument(zookeeperSyncTimeMillis >= 0, "The ZK sync time must be positive");
-
-		this.zookeeperAddress = zookeeperAddress;
-		this.groupId = groupId;
-		this.topicId = topicId;
-		this.zookeeperSyncTimeMillis = zookeeperSyncTimeMillis;
-		this.customProperties = customProperties;
-	}
-
-	/**
-	 * Creates a KafkaSource that consumes a topic.
-	 *
-	 * @param zookeeperAddress
-	 *            Address of the Zookeeper host (with port number).
-	 * @param topicId
-	 *            ID of the Kafka topic.
-	 * @param deserializationSchema
-	 *            User defined deserialization schema.
-	 * @param zookeeperSyncTimeMillis
-	 *            Synchronization time with zookeeper.
-	 */
-	public KafkaSource(String zookeeperAddress, String topicId, DeserializationSchema<OUT> deserializationSchema, long zookeeperSyncTimeMillis) {
-		this(zookeeperAddress, topicId, DEFAULT_GROUP_ID, deserializationSchema, zookeeperSyncTimeMillis, null);
-	}
-	/**
-	 * Creates a KafkaSource that consumes a topic.
-	 *
-	 * @param zookeeperAddress
-	 *            Address of the Zookeeper host (with port number).
-	 * @param topicId
-	 *            ID of the Kafka topic.
-	 * @param deserializationSchema
-	 *            User defined deserialization schema.
-	 */
-	public KafkaSource(String zookeeperAddress, String topicId, DeserializationSchema<OUT> deserializationSchema) {
-		this(zookeeperAddress, topicId, deserializationSchema, ZOOKEEPER_DEFAULT_SYNC_TIME);
-	}
-
-	/**
-	 * Initializes the connection to Kafka.
-	 */
-	private void initializeConnection() {
-		Properties props = new Properties();
-		props.put("zookeeper.connect", zookeeperAddress);
-		props.put("group.id", groupId);
-		props.put("zookeeper.session.timeout.ms", "10000");
-		props.put("zookeeper.sync.time.ms", Long.toString(zookeeperSyncTimeMillis));
-		props.put("auto.commit.interval.ms", "1000");
-
-		if (customProperties != null) {
-			for(Map.Entry<Object, Object> e : props.entrySet()) {
-				if(props.contains(e.getKey())) {
-					LOG.warn("Overwriting property "+e.getKey()+" with value "+e.getValue());
-				}
-				props.put(e.getKey(), e.getValue());
-			}
-		}
-
-		consumer = Consumer.createJavaConsumerConnector(new ConsumerConfig(props));
-
-		Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer
-				.createMessageStreams(Collections.singletonMap(topicId, 1));
-		List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topicId);
-		KafkaStream<byte[], byte[]> stream = streams.get(0);
-
-		consumer.commitOffsets();
-
-		consumerIterator = stream.iterator();
-	}
-
-	@Override
-	public void run(SourceContext<OUT> ctx) throws Exception {
-		
-		// NOTE: Since this source is not checkpointed, we do not need to
-		// acquire the checkpoint lock
-		try {
-			while (isRunning && consumerIterator.hasNext()) {
-				OUT out = schema.deserialize(consumerIterator.next().message());
-				if (schema.isEndOfStream(out)) {
-					break;
-				}
-				ctx.collect(out);
-			}
-		} finally {
-			consumer.shutdown();
-		}
-	}
-
-	@Override
-	public void open(Configuration config) throws Exception {
-		initializeConnection();
-		isRunning = true;
-	}
-
-	@Override
-	public void cancel() {
-		isRunning = false;
-		if (consumer != null) {
-			consumer.shutdown();
-		}
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
index 042fcdf..4181134 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
@@ -17,370 +17,36 @@
 
 package org.apache.flink.streaming.connectors.kafka.api.persistent;
 
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.UnsupportedEncodingException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import kafka.common.TopicAndPartition;
-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.utils.ZKGroupTopicDirs;
-import kafka.utils.ZkUtils;
-
-import org.I0Itec.zkclient.ZkClient;
-import org.I0Itec.zkclient.exception.ZkMarshallingError;
-import org.I0Itec.zkclient.serialize.ZkSerializer;
-import org.apache.commons.collections.map.LinkedMap;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.connectors.FlinkKafkaConsumer;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import scala.Option;
-import scala.collection.JavaConversions;
-import scala.collection.Seq;
 
-import com.google.common.base.Preconditions;
 
 /**
- * Source for reading from Kafka using Flink Streaming Fault Tolerance.
- * This source is updating the committed offset in Zookeeper based on the internal checkpointing of Flink.
+ * Creates a Kafka consumer compatible with reading from Kafka 0.8.1+ consumers.
+ *
+ * This class is provided as a migration path from the old Flink kafka connectors to the new, updated implemntations.
+ *
+ * Please use FlinkKafkaConsumer081 and FlinkKafkaConsumer082.
  *
- * Note that the autocommit feature of Kafka needs to be disabled for using this source.
+ * @param <T> The type of elements produced by this consumer.
  */
-public class PersistentKafkaSource<OUT> extends RichParallelSourceFunction<OUT> implements
-		ResultTypeQueryable<OUT>,
-		CheckpointNotifier, CheckpointedAsynchronously<long[]> {
+@Deprecated
+public class PersistentKafkaSource<T> extends FlinkKafkaConsumer<T> {
 
-	private static final long serialVersionUID = 287845877188312621L;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(PersistentKafkaSource.class);
+	private static final long serialVersionUID = -8450689820627198228L;
 
-	private final LinkedMap pendingCheckpoints = new LinkedMap();
-
-	private final String topicName;
-	private final DeserializationSchema<OUT> deserializationSchema;
-	
-	private transient ConsumerConfig consumerConfig;
-	private transient ConsumerIterator<byte[], byte[]> iteratorToRead;
-	private transient ConsumerConnector consumer;
-	
-	private transient ZkClient zkClient;
-
-	private transient long[] lastOffsets;			// Current offset (backuped state)
-	protected transient long[] commitedOffsets; 	// maintain committed offsets, to avoid committing the same over and over again.
-	private transient long[] restoreState;			// set by the restore() method, used by open() to valdiate the restored state.
-
-	private volatile boolean running;
-	
 	/**
+	 * Creates a new Kafka 0.8.2.x streaming source consumer.
 	 *
-	 * For the @param consumerConfig, specify at least the "groupid" and "zookeeper.connect" string.
-	 * The config will be passed into the Kafka High Level Consumer.
-	 * For a full list of possible values, check this out: https://kafka.apache.org/documentation.html#consumerconfigs
-	 */
-	public PersistentKafkaSource(String topicName, DeserializationSchema<OUT> deserializationSchema, ConsumerConfig consumerConfig) {
-		Preconditions.checkNotNull(topicName);
-		Preconditions.checkNotNull(deserializationSchema);
-		Preconditions.checkNotNull(consumerConfig);
-
-		this.topicName = topicName;
-		this.deserializationSchema = deserializationSchema;
-		this.consumerConfig = consumerConfig;
-		if (consumerConfig.autoCommitEnable()) {
-			throw new IllegalArgumentException("'auto.commit.enable' is set to 'true'. " +
-					"This source can only be used with auto commit disabled because the " +
-					"source is committing to zookeeper by itself (not using the KafkaConsumer).");
-		}
-		if (!consumerConfig.offsetsStorage().equals("zookeeper")) {
-			// we can currently only commit to ZK.
-			throw new IllegalArgumentException("The 'offsets.storage' has to be set to 'zookeeper' for this Source to work reliably");
-		}
-	}
-
-	// ---------------------- ParallelSourceFunction Lifecycle -----------------
-
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		ConsumerConnector consumer = Consumer.createJavaConsumerConnector(this.consumerConfig);
-		// 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 = consumer.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, consumerConfig.groupId());
-		this.iteratorToRead = kafkaStreams.get(0).iterator();
-		this.consumer = consumer;
-
-		zkClient = new ZkClient(consumerConfig.zkConnect(),
-				consumerConfig.zkSessionTimeoutMs(),
-				consumerConfig.zkConnectionTimeoutMs(),
-				new KafkaZKStringSerializer());
-
-		// most likely the number of offsets we're going to store here will be lower than the number of partitions.
-		int numPartitions = getNumberOfPartitions();
-		LOG.debug("The topic {} has {} partitions", topicName, numPartitions);
-		this.lastOffsets = new long[numPartitions];
-		this.commitedOffsets = new long[numPartitions];
-
-		// check if there are offsets to restore
-		if (restoreState != null) {
-			if (restoreState.length != numPartitions) {
-				throw new IllegalStateException("There are "+restoreState.length+" offsets to restore for topic "+topicName+" but " +
-						"there are only "+numPartitions+" in the topic");
-			}
-
-			LOG.info("Setting restored offsets {} in ZooKeeper", Arrays.toString(restoreState));
-			setOffsetsInZooKeeper(restoreState);
-			this.lastOffsets = restoreState;
-		} else {
-			// initialize empty offsets
-			Arrays.fill(this.lastOffsets, -1);
-		}
-		Arrays.fill(this.commitedOffsets, 0); // just to make it clear
-
-		pendingCheckpoints.clear();
-		running = true;
-	}
-
-	@Override
-	public void run(SourceContext<OUT> ctx) throws Exception {
-		if (iteratorToRead == null) {
-			throw new IllegalStateException("Kafka iterator not initialized properly.");
-		}
-
-		final Object checkpointLock = ctx.getCheckpointLock();
-		
-		while (running && iteratorToRead.hasNext()) {
-			MessageAndMetadata<byte[], byte[]> message = iteratorToRead.next();
-			if(lastOffsets[message.partition()] >= message.offset()) {
-				LOG.info("Skipping message with offset {} from partition {}", message.offset(), message.partition());
-				continue;
-			}
-			OUT next = deserializationSchema.deserialize(message.message());
-
-			if (deserializationSchema.isEndOfStream(next)) {
-				LOG.info("DeserializationSchema signaled end of stream for this source");
-				break;
-			}
-
-			// make the state update and the element emission atomic
-			synchronized (checkpointLock) {
-				lastOffsets[message.partition()] = message.offset();
-				ctx.collect(next);
-			}
-
-			if (LOG.isTraceEnabled()) {
-				LOG.trace("Processed record with offset {} from partition {}", message.offset(), message.partition());
-			}
-		}
-	}
-
-	@Override
-	public void cancel() {
-		running = false;
-	}
-
-	@Override
-	public void close() {
-		LOG.info("Closing Kafka consumer");
-		this.consumer.shutdown();
-		zkClient.close();
-	}
-
-	// -----------------  State Checkpointing -----------------
-
-	@Override
-	public long[] snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-		if (lastOffsets == null) {
-			LOG.warn("State snapshot requested on not yet opened source. Returning null");
-			return null;
-		}
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Snapshotting state. Offsets: {}, checkpoint id {}, timestamp {}",
-					Arrays.toString(lastOffsets), checkpointId, checkpointTimestamp);
-		}
-
-		long[] currentOffsets = Arrays.copyOf(lastOffsets, lastOffsets.length);
-
-		// the map may be asynchronously updates when committing to Kafka, so we synchronize
-		synchronized (pendingCheckpoints) {
-			pendingCheckpoints.put(checkpointId, currentOffsets);
-		}
-
-		return currentOffsets;
-	}
-
-	@Override
-	public void restoreState(long[] state) {
-		LOG.info("The state will be restored to {} in the open() method", Arrays.toString(state));
-		this.restoreState = Arrays.copyOf(state, state.length);
-	}
-
-	
-	/**
-	 * Notification on completed checkpoints
-	 * @param checkpointId The ID of the checkpoint that has been completed.
-	 * @throws Exception 
-	 */
-	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		LOG.info("Commit checkpoint {}", checkpointId);
-
-		long[] checkpointOffsets;
-
-		// the map may be asynchronously updates when snapshotting state, so we synchronize
-		synchronized (pendingCheckpoints) {
-			final int posInMap = pendingCheckpoints.indexOf(checkpointId);
-			if (posInMap == -1) {
-				LOG.warn("Unable to find pending checkpoint for id {}", checkpointId);
-				return;
-			}
-
-			checkpointOffsets = (long[]) pendingCheckpoints.remove(posInMap);
-			// remove older checkpoints in map:
-			if (!pendingCheckpoints.isEmpty()) {
-				for(int i = 0; i < posInMap; i++) {
-					pendingCheckpoints.remove(0);
-				}
-			}
-		}
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Committing offsets {} to ZooKeeper", Arrays.toString(checkpointOffsets));
-		}
-
-		setOffsetsInZooKeeper(checkpointOffsets);
-	}
-
-	private void setOffsetsInZooKeeper(long[] offsets) {
-		for (int partition = 0; partition < offsets.length; partition++) {
-			long offset = offsets[partition];
-			if(offset != -1) {
-				setOffset(partition, offset);
-			}
-		}
-	}
-
-	// --------------------- Zookeeper / Offset handling -----------------------------
-
-	private int getNumberOfPartitions() {
-		scala.collection.immutable.List<String> scalaSeq = JavaConversions.asScalaBuffer(Collections.singletonList(topicName)).toList();
-		scala.collection.mutable.Map<String, Seq<Object>> list =  ZkUtils.getPartitionsForTopics(zkClient, scalaSeq);
-		Option<Seq<Object>> topicOption = list.get(topicName);
-		if(topicOption.isEmpty()) {
-			throw new IllegalArgumentException("Unable to get number of partitions for topic "+topicName+" from "+list.toString());
-		}
-		Seq<Object> topic = topicOption.get();
-		return topic.size();
-	}
-
-	protected void setOffset(int partition, long offset) {
-		// synchronize because notifyCheckpointComplete is called using asynchronous worker threads (= multiple checkpoints might be confirmed concurrently)
-		synchronized (commitedOffsets) {
-			if(commitedOffsets[partition] < offset) {
-				setOffset(zkClient, consumerConfig.groupId(), topicName, partition, offset);
-				commitedOffsets[partition] = offset;
-			} else {
-				LOG.debug("Ignoring offset {} for partition {} because it is already committed", offset, partition);
-			}
-		}
-	}
-
-
-
-	// the following two methods are static to allow access from the outside as well (Testcases)
-
-	/**
-	 * This method's code is based on ZookeeperConsumerConnector.commitOffsetToZooKeeper()
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param consumerConfig
+	 *           The consumer config used to configure the Kafka consumer client, and the ZooKeeper client.
 	 */
-	public static void setOffset(ZkClient zkClient, String groupId, String topic, int partition, long offset) {
-		LOG.info("Setting offset for partition {} of topic {} in group {} to offset {}", partition, topic, groupId, offset);
-		TopicAndPartition tap = new TopicAndPartition(topic, partition);
-		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, tap.topic());
-		ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir() + "/" + tap.partition(), Long.toString(offset));
-	}
-
-	public static long getOffset(ZkClient zkClient, String groupId, String topic, int partition) {
-		TopicAndPartition tap = new TopicAndPartition(topic, partition);
-		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, tap.topic());
-		scala.Tuple2<String, Stat> data = ZkUtils.readData(zkClient, topicDirs.consumerOffsetDir() + "/" + tap.partition());
-		return Long.valueOf(data._1());
-	}
-
-
-	// ---------------------- (Java)Serialization methods for the consumerConfig -----------------
-
-	private void writeObject(ObjectOutputStream out)
-			throws IOException, ClassNotFoundException {
-		out.defaultWriteObject();
-		out.writeObject(consumerConfig.props().props());
-	}
-
-	private void readObject(ObjectInputStream in)
-			throws IOException, ClassNotFoundException {
-		in.defaultReadObject();
-		Properties props = (Properties) in.readObject();
-		consumerConfig = new ConsumerConfig(props);
-	}
-
-
-	@Override
-	public TypeInformation<OUT> getProducedType() {
-		return deserializationSchema.getProducedType();
-	}
-
-	// ---------------------- Zookeeper Serializer copied from Kafka (because it has private access there)  -----------------
-
-	public static class KafkaZKStringSerializer implements ZkSerializer {
-
-		@Override
-		public byte[] serialize(Object data) throws ZkMarshallingError {
-			try {
-				return ((String) data).getBytes("UTF-8");
-			} catch (UnsupportedEncodingException e) {
-				throw new RuntimeException(e);
-			}
-		}
-
-		@Override
-		public Object deserialize(byte[] bytes) throws ZkMarshallingError {
-			if (bytes == null) {
-				return null;
-			} else {
-				try {
-					return new String(bytes, "UTF-8");
-				} catch (UnsupportedEncodingException e) {
-					throw new RuntimeException(e);
-				}
-			}
-		}
+	public PersistentKafkaSource(String topic, DeserializationSchema<T> valueDeserializer, ConsumerConfig consumerConfig) {
+		super(topic, valueDeserializer, consumerConfig.props().props(), OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaConstantPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaConstantPartitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaConstantPartitioner.java
deleted file mode 100644
index 661d0bd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaConstantPartitioner.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.partitioner;
-
-public class KafkaConstantPartitioner implements SerializableKafkaPartitioner {
-
-	private static final long serialVersionUID = 1L;
-	private int partition;
-
-	public KafkaConstantPartitioner(int partition) {
-		this.partition = partition;
-	}
-
-	@Override
-	public int partition(Object value, int numberOfPartitions) {
-		return partition;
-	}
-}

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

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


[18/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MeasurableStat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MeasurableStat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MeasurableStat.java
new file mode 100644
index 0000000..a3ee306
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MeasurableStat.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.kafka_backport.common.metrics;
+
+import org.apache.flink.kafka_backport.common.metrics.stats.Count;
+import org.apache.flink.kafka_backport.common.metrics.stats.Max;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A MeasurableStat is a {@link Stat} that is also {@link Measurable} (i.e. can produce a single floating point value).
+ * This is the interface used for most of the simple statistics such as {@link org.apache.flink.kafka_backport.common.metrics.stats.Avg},
+ * {@link Max}, {@link Count}, etc.
+ */
+public interface MeasurableStat extends Stat, Measurable {
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricConfig.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricConfig.java
new file mode 100644
index 0000000..f95824a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricConfig.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.kafka_backport.common.metrics;
+
+import java.util.concurrent.TimeUnit;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Configuration values for metrics
+ */
+public class MetricConfig {
+
+    private Quota quota;
+    private int samples;
+    private long eventWindow;
+    private long timeWindowMs;
+    private TimeUnit unit;
+
+    public MetricConfig() {
+        super();
+        this.quota = null;
+        this.samples = 2;
+        this.eventWindow = Long.MAX_VALUE;
+        this.timeWindowMs = TimeUnit.MILLISECONDS.convert(30, TimeUnit.SECONDS);
+        this.unit = TimeUnit.SECONDS;
+    }
+
+    public Quota quota() {
+        return this.quota;
+    }
+
+    public MetricConfig quota(Quota quota) {
+        this.quota = quota;
+        return this;
+    }
+
+    public long eventWindow() {
+        return eventWindow;
+    }
+
+    public MetricConfig eventWindow(long window) {
+        this.eventWindow = window;
+        return this;
+    }
+
+    public long timeWindowMs() {
+        return timeWindowMs;
+    }
+
+    public MetricConfig timeWindow(long window, TimeUnit unit) {
+        this.timeWindowMs = TimeUnit.MILLISECONDS.convert(window, unit);
+        return this;
+    }
+
+    public int samples() {
+        return this.samples;
+    }
+
+    public MetricConfig samples(int samples) {
+        if (samples < 1)
+            throw new IllegalArgumentException("The number of samples must be at least 1.");
+        this.samples = samples;
+        return this;
+    }
+
+    public TimeUnit timeUnit() {
+        return unit;
+    }
+
+    public MetricConfig timeUnit(TimeUnit unit) {
+        this.unit = unit;
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Metrics.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Metrics.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Metrics.java
new file mode 100644
index 0000000..ea7e5f0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Metrics.java
@@ -0,0 +1,211 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.metrics;
+
+import org.apache.flink.kafka_backport.common.utils.SystemTime;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.apache.flink.kafka_backport.common.utils.CopyOnWriteMap;
+import org.apache.flink.kafka_backport.common.utils.Time;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A registry of sensors and metrics.
+ * <p>
+ * A metric is a named, numerical measurement. A sensor is a handle to record numerical measurements as they occur. Each
+ * Sensor has zero or more associated metrics. For example a Sensor might represent message sizes and we might associate
+ * with this sensor a metric for the average, maximum, or other statistics computed off the sequence of message sizes
+ * that are recorded by the sensor.
+ * <p>
+ * Usage looks something like this:
+ * 
+ * <pre>
+ * // set up metrics:
+ * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
+ * Sensor sensor = metrics.sensor(&quot;message-sizes&quot;);
+ * MetricName metricName = new MetricName(&quot;message-size-avg&quot;, &quot;producer-metrics&quot;);
+ * sensor.add(metricName, new Avg());
+ * metricName = new MetricName(&quot;message-size-max&quot;, &quot;producer-metrics&quot;);
+ * sensor.add(metricName, new Max());
+ * 
+ * // as messages are sent we record the sizes
+ * sensor.record(messageSize);
+ * </pre>
+ */
+public class Metrics implements Closeable {
+
+    private final MetricConfig config;
+    private final ConcurrentMap<MetricName, KafkaMetric> metrics;
+    private final ConcurrentMap<String, Sensor> sensors;
+    private final List<MetricsReporter> reporters;
+    private final Time time;
+
+    /**
+     * Create a metrics repository with no metric reporters and default configuration.
+     */
+    public Metrics() {
+        this(new MetricConfig());
+    }
+
+    /**
+     * Create a metrics repository with no metric reporters and default configuration.
+     */
+    public Metrics(Time time) {
+        this(new MetricConfig(), new ArrayList<MetricsReporter>(0), time);
+    }
+
+    /**
+     * Create a metrics repository with no reporters and the given default config. This config will be used for any
+     * metric that doesn't override its own config.
+     * @param defaultConfig The default config to use for all metrics that don't override their config
+     */
+    public Metrics(MetricConfig defaultConfig) {
+        this(defaultConfig, new ArrayList<MetricsReporter>(0), new SystemTime());
+    }
+
+    /**
+     * Create a metrics repository with a default config and the given metric reporters
+     * @param defaultConfig The default config
+     * @param reporters The metrics reporters
+     * @param time The time instance to use with the metrics
+     */
+    public Metrics(MetricConfig defaultConfig, List<MetricsReporter> reporters, Time time) {
+        this.config = defaultConfig;
+        this.sensors = new CopyOnWriteMap<String, Sensor>();
+        this.metrics = new CopyOnWriteMap<MetricName, KafkaMetric>();
+        this.reporters = Utils.notNull(reporters);
+        this.time = time;
+        for (MetricsReporter reporter : reporters)
+            reporter.init(new ArrayList<KafkaMetric>());
+    }
+
+    /**
+     * Get the sensor with the given name if it exists
+     * @param name The name of the sensor
+     * @return Return the sensor or null if no such sensor exists
+     */
+    public Sensor getSensor(String name) {
+        return this.sensors.get(Utils.notNull(name));
+    }
+
+    /**
+     * Get or create a sensor with the given unique name and no parent sensors.
+     * @param name The sensor name
+     * @return The sensor
+     */
+    public Sensor sensor(String name) {
+        return sensor(name, null, (Sensor[]) null);
+    }
+
+    /**
+     * Get or create a sensor with the given unique name and zero or more parent sensors. All parent sensors will
+     * receive every value recorded with this sensor.
+     * @param name The name of the sensor
+     * @param parents The parent sensors
+     * @return The sensor that is created
+     */
+    public Sensor sensor(String name, Sensor... parents) {
+        return sensor(name, null, parents);
+    }
+
+    /**
+     * Get or create a sensor with the given unique name and zero or more parent sensors. All parent sensors will
+     * receive every value recorded with this sensor.
+     * @param name The name of the sensor
+     * @param config A default configuration to use for this sensor for metrics that don't have their own config
+     * @param parents The parent sensors
+     * @return The sensor that is created
+     */
+    public synchronized Sensor sensor(String name, MetricConfig config, Sensor... parents) {
+        Sensor s = getSensor(name);
+        if (s == null) {
+            s = new Sensor(this, name, parents, config == null ? this.config : config, time);
+            this.sensors.put(name, s);
+        }
+        return s;
+    }
+
+    /**
+     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
+     * This is a way to expose existing values as metrics.
+     * @param metricName The name of the metric
+     * @param measurable The measurable that will be measured by this metric
+     */
+    public void addMetric(MetricName metricName, Measurable measurable) {
+        addMetric(metricName, null, measurable);
+    }
+
+    /**
+     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
+     * This is a way to expose existing values as metrics.
+     * @param metricName The name of the metric
+     * @param config The configuration to use when measuring this measurable
+     * @param measurable The measurable that will be measured by this metric
+     */
+    public synchronized void addMetric(MetricName metricName, MetricConfig config, Measurable measurable) {
+        KafkaMetric m = new KafkaMetric(new Object(),
+                                        Utils.notNull(metricName),
+                                        Utils.notNull(measurable),
+                                        config == null ? this.config : config,
+                                        time);
+        registerMetric(m);
+    }
+
+    /**
+     * Add a MetricReporter
+     */
+    public synchronized void addReporter(MetricsReporter reporter) {
+        Utils.notNull(reporter).init(new ArrayList<KafkaMetric>(metrics.values()));
+        this.reporters.add(reporter);
+    }
+
+    synchronized void registerMetric(KafkaMetric metric) {
+        MetricName metricName = metric.metricName();
+        if (this.metrics.containsKey(metricName))
+            throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one.");
+        this.metrics.put(metricName, metric);
+        for (MetricsReporter reporter : reporters)
+            reporter.metricChange(metric);
+    }
+
+    /**
+     * Get all the metrics currently maintained indexed by metricName
+     */
+    public Map<MetricName, KafkaMetric> metrics() {
+        return this.metrics;
+    }
+
+    /**
+     * Close this metrics repository.
+     */
+    @Override
+    public void close() {
+        for (MetricsReporter reporter : this.reporters)
+            reporter.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricsReporter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricsReporter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricsReporter.java
new file mode 100644
index 0000000..870198b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricsReporter.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.kafka_backport.common.metrics;
+
+import org.apache.flink.kafka_backport.common.Configurable;
+
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A plugin interface to allow things to listen as new metrics are created so they can be reported.
+ */
+public interface MetricsReporter extends Configurable {
+
+    /**
+     * This is called when the reporter is first registered to initially register all existing metrics
+     * @param metrics All currently existing metrics
+     */
+    public void init(List<KafkaMetric> metrics);
+
+    /**
+     * This is called whenever a metric is updated or added
+     * @param metric
+     */
+    public void metricChange(KafkaMetric metric);
+
+    /**
+     * Called when the metrics repository is closed.
+     */
+    public void close();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Quota.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Quota.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Quota.java
new file mode 100644
index 0000000..1de74bc
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Quota.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.metrics;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * An upper or lower bound for metrics
+ */
+public final class Quota {
+
+    private final boolean upper;
+    private final double bound;
+
+    public Quota(double bound, boolean upper) {
+        this.bound = bound;
+        this.upper = upper;
+    }
+
+    public static Quota lessThan(double upperBound) {
+        return new Quota(upperBound, true);
+    }
+
+    public static Quota moreThan(double lowerBound) {
+        return new Quota(lowerBound, false);
+    }
+
+    public boolean isUpperBound() {
+        return this.upper;
+    }
+
+    public double bound() {
+        return this.bound;
+    }
+
+    public boolean acceptable(double value) {
+        return (upper && value <= bound) || (!upper && value >= bound);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/QuotaViolationException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/QuotaViolationException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/QuotaViolationException.java
new file mode 100644
index 0000000..ee1d073
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/QuotaViolationException.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.kafka_backport.common.metrics;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Thrown when a sensor records a value that causes a metric to go outside the bounds configured as its quota
+ */
+public class QuotaViolationException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public QuotaViolationException(String m) {
+        super(m);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Sensor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Sensor.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Sensor.java
new file mode 100644
index 0000000..16be7ec
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Sensor.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.kafka_backport.common.metrics;
+
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.apache.flink.kafka_backport.common.utils.Time;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A sensor applies a continuous sequence of numerical values to a set of associated metrics. For example a sensor on
+ * message size would record a sequence of message sizes using the {@link #record(double)} api and would maintain a set
+ * of metrics about request sizes such as the average or max.
+ */
+public final class Sensor {
+
+    private final Metrics registry;
+    private final String name;
+    private final Sensor[] parents;
+    private final List<Stat> stats;
+    private final List<KafkaMetric> metrics;
+    private final MetricConfig config;
+    private final Time time;
+
+    Sensor(Metrics registry, String name, Sensor[] parents, MetricConfig config, Time time) {
+        super();
+        this.registry = registry;
+        this.name = Utils.notNull(name);
+        this.parents = parents == null ? new Sensor[0] : parents;
+        this.metrics = new ArrayList<KafkaMetric>();
+        this.stats = new ArrayList<Stat>();
+        this.config = config;
+        this.time = time;
+        checkForest(new HashSet<Sensor>());
+    }
+
+    /* Validate that this sensor doesn't end up referencing itself */
+    private void checkForest(Set<Sensor> sensors) {
+        if (!sensors.add(this))
+            throw new IllegalArgumentException("Circular dependency in sensors: " + name() + " is its own parent.");
+        for (int i = 0; i < parents.length; i++)
+            parents[i].checkForest(sensors);
+    }
+
+    /**
+     * The name this sensor is registered with. This name will be unique among all registered sensors.
+     */
+    public String name() {
+        return this.name;
+    }
+
+    /**
+     * Record an occurrence, this is just short-hand for {@link #record(double) record(1.0)}
+     */
+    public void record() {
+        record(1.0);
+    }
+
+    /**
+     * Record a value with this sensor
+     * @param value The value to record
+     * @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum
+     *         bound
+     */
+    public void record(double value) {
+        record(value, time.milliseconds());
+    }
+
+    /**
+     * Record a value at a known time. This method is slightly faster than {@link #record(double)} since it will reuse
+     * the time stamp.
+     * @param value The value we are recording
+     * @param timeMs The current POSIX time in milliseconds
+     * @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum
+     *         bound
+     */
+    public void record(double value, long timeMs) {
+        synchronized (this) {
+            // increment all the stats
+            for (int i = 0; i < this.stats.size(); i++)
+                this.stats.get(i).record(config, value, timeMs);
+            checkQuotas(timeMs);
+        }
+        for (int i = 0; i < parents.length; i++)
+            parents[i].record(value, timeMs);
+    }
+
+    /**
+     * Check if we have violated our quota for any metric that has a configured quota
+     * @param timeMs
+     */
+    private void checkQuotas(long timeMs) {
+        for (int i = 0; i < this.metrics.size(); i++) {
+            KafkaMetric metric = this.metrics.get(i);
+            MetricConfig config = metric.config();
+            if (config != null) {
+                Quota quota = config.quota();
+                if (quota != null) {
+                    if (!quota.acceptable(metric.value(timeMs)))
+                        throw new QuotaViolationException(metric.metricName() + " is in violation of its quota of " + quota.bound());
+                }
+            }
+        }
+    }
+
+    /**
+     * Register a compound statistic with this sensor with no config override
+     */
+    public void add(CompoundStat stat) {
+        add(stat, null);
+    }
+
+    /**
+     * Register a compound statistic with this sensor which yields multiple measurable quantities (like a histogram)
+     * @param stat The stat to register
+     * @param config The configuration for this stat. If null then the stat will use the default configuration for this
+     *        sensor.
+     */
+    public synchronized void add(CompoundStat stat, MetricConfig config) {
+        this.stats.add(Utils.notNull(stat));
+        for (CompoundStat.NamedMeasurable m : stat.stats()) {
+            KafkaMetric metric = new KafkaMetric(this, m.name(), m.stat(), config == null ? this.config : config, time);
+            this.registry.registerMetric(metric);
+            this.metrics.add(metric);
+        }
+    }
+
+    /**
+     * Register a metric with this sensor
+     * @param metricName The name of the metric
+     * @param stat The statistic to keep
+     */
+    public void add(MetricName metricName, MeasurableStat stat) {
+        add(metricName, stat, null);
+    }
+
+    /**
+     * Register a metric with this sensor
+     * @param metricName The name of the metric
+     * @param stat The statistic to keep
+     * @param config A special configuration for this metric. If null use the sensor default configuration.
+     */
+    public synchronized void add(MetricName metricName, MeasurableStat stat, MetricConfig config) {
+        KafkaMetric metric = new KafkaMetric(new Object(),
+                                             Utils.notNull(metricName),
+                                             Utils.notNull(stat),
+                                             config == null ? this.config : config,
+                                             time);
+        this.registry.registerMetric(metric);
+        this.metrics.add(metric);
+        this.stats.add(stat);
+    }
+
+    synchronized List<KafkaMetric> metrics() {
+        return Collections.unmodifiableList(this.metrics);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Stat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Stat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Stat.java
new file mode 100644
index 0000000..25ce62b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Stat.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.kafka_backport.common.metrics;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A Stat is a quanity such as average, max, etc that is computed off the stream of updates to a sensor
+ */
+public interface Stat {
+
+    /**
+     * Record the given value
+     * @param config The configuration to use for this metric
+     * @param value The value to record
+     * @param timeMs The POSIX time in milliseconds this value occurred
+     */
+    public void record(MetricConfig config, double value, long timeMs);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Avg.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Avg.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Avg.java
new file mode 100644
index 0000000..6b58bc7
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Avg.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.kafka_backport.common.metrics.stats;
+
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A {@link SampledStat} that maintains a simple average over its samples.
+ */
+public class Avg extends SampledStat {
+
+    public Avg() {
+        super(0.0);
+    }
+
+    @Override
+    protected void update(Sample sample, MetricConfig config, double value, long now) {
+        sample.value += value;
+    }
+
+    @Override
+    public double combine(List<Sample> samples, MetricConfig config, long now) {
+        double total = 0.0;
+        long count = 0;
+        for (int i = 0; i < samples.size(); i++) {
+            Sample s = samples.get(i);
+            total += s.value;
+            count += s.eventCount;
+        }
+        return total / count;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Count.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Count.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Count.java
new file mode 100644
index 0000000..b7fdb1e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Count.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.kafka_backport.common.metrics.stats;
+
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A {@link SampledStat} that maintains a simple count of what it has seen.
+ */
+public class Count extends SampledStat {
+
+    public Count() {
+        super(0);
+    }
+
+    @Override
+    protected void update(Sample sample, MetricConfig config, double value, long now) {
+        sample.value += 1.0;
+    }
+
+    @Override
+    public double combine(List<Sample> samples, MetricConfig config, long now) {
+        double total = 0.0;
+        for (int i = 0; i < samples.size(); i++)
+            total += samples.get(i).value;
+        return total;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Histogram.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Histogram.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Histogram.java
new file mode 100644
index 0000000..9289e89
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Histogram.java
@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.metrics.stats;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class Histogram {
+
+    private final BinScheme binScheme;
+    private final float[] hist;
+    private double count;
+
+    public Histogram(BinScheme binScheme) {
+        this.hist = new float[binScheme.bins()];
+        this.count = 0.0f;
+        this.binScheme = binScheme;
+    }
+
+    public void record(double value) {
+        this.hist[binScheme.toBin(value)] += 1.0f;
+        this.count += 1.0f;
+    }
+
+    public double value(double quantile) {
+        if (count == 0.0d)
+            return Double.NaN;
+        float sum = 0.0f;
+        float quant = (float) quantile;
+        for (int i = 0; i < this.hist.length - 1; i++) {
+            sum += this.hist[i];
+            if (sum / count > quant)
+                return binScheme.fromBin(i);
+        }
+        return Float.POSITIVE_INFINITY;
+    }
+
+    public float[] counts() {
+        return this.hist;
+    }
+
+    public void clear() {
+        for (int i = 0; i < this.hist.length; i++)
+            this.hist[i] = 0.0f;
+        this.count = 0;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder b = new StringBuilder("{");
+        for (int i = 0; i < this.hist.length - 1; i++) {
+            b.append(String.format("%.10f", binScheme.fromBin(i)));
+            b.append(':');
+            b.append(String.format("%.0f", this.hist[i]));
+            b.append(',');
+        }
+        b.append(Float.POSITIVE_INFINITY);
+        b.append(':');
+        b.append(this.hist[this.hist.length - 1]);
+        b.append('}');
+        return b.toString();
+    }
+
+    public interface BinScheme {
+        public int bins();
+
+        public int toBin(double value);
+
+        public double fromBin(int bin);
+    }
+
+    public static class ConstantBinScheme implements BinScheme {
+        private final double min;
+        private final double max;
+        private final int bins;
+        private final double bucketWidth;
+
+        public ConstantBinScheme(int bins, double min, double max) {
+            if (bins < 2)
+                throw new IllegalArgumentException("Must have at least 2 bins.");
+            this.min = min;
+            this.max = max;
+            this.bins = bins;
+            this.bucketWidth = (max - min) / (bins - 2);
+        }
+
+        public int bins() {
+            return this.bins;
+        }
+
+        public double fromBin(int b) {
+            if (b == 0)
+                return Double.NEGATIVE_INFINITY;
+            else if (b == bins - 1)
+                return Double.POSITIVE_INFINITY;
+            else
+                return min + (b - 1) * bucketWidth;
+        }
+
+        public int toBin(double x) {
+            if (x < min)
+                return 0;
+            else if (x > max)
+                return bins - 1;
+            else
+                return (int) ((x - min) / bucketWidth) + 1;
+        }
+    }
+
+    public static class LinearBinScheme implements BinScheme {
+        private final int bins;
+        private final double max;
+        private final double scale;
+
+        public LinearBinScheme(int numBins, double max) {
+            this.bins = numBins;
+            this.max = max;
+            this.scale = max / (numBins * (numBins - 1) / 2);
+        }
+
+        public int bins() {
+            return this.bins;
+        }
+
+        public double fromBin(int b) {
+            if (b == this.bins - 1) {
+                return Float.POSITIVE_INFINITY;
+            } else {
+                double unscaled = (b * (b + 1.0)) / 2.0;
+                return unscaled * this.scale;
+            }
+        }
+
+        public int toBin(double x) {
+            if (x < 0.0d) {
+                throw new IllegalArgumentException("Values less than 0.0 not accepted.");
+            } else if (x > this.max) {
+                return this.bins - 1;
+            } else {
+                double scaled = x / this.scale;
+                return (int) (-0.5 + Math.sqrt(2.0 * scaled + 0.25));
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Max.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Max.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Max.java
new file mode 100644
index 0000000..82390eb
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Max.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.kafka_backport.common.metrics.stats;
+
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A {@link SampledStat} that gives the max over its samples.
+ */
+public final class Max extends SampledStat {
+
+    public Max() {
+        super(Double.NEGATIVE_INFINITY);
+    }
+
+    @Override
+    protected void update(Sample sample, MetricConfig config, double value, long now) {
+        sample.value = Math.max(sample.value, value);
+    }
+
+    @Override
+    public double combine(List<Sample> samples, MetricConfig config, long now) {
+        double max = Double.NEGATIVE_INFINITY;
+        for (int i = 0; i < samples.size(); i++)
+            max = Math.max(max, samples.get(i).value);
+        return max;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Min.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Min.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Min.java
new file mode 100644
index 0000000..88fbe79
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Min.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.kafka_backport.common.metrics.stats;
+
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A {@link SampledStat} that gives the min over its samples.
+ */
+public class Min extends SampledStat {
+
+    public Min() {
+        super(Double.MIN_VALUE);
+    }
+
+    @Override
+    protected void update(Sample sample, MetricConfig config, double value, long now) {
+        sample.value = Math.min(sample.value, value);
+    }
+
+    @Override
+    public double combine(List<Sample> samples, MetricConfig config, long now) {
+        double max = Double.MAX_VALUE;
+        for (int i = 0; i < samples.size(); i++)
+            max = Math.min(max, samples.get(i).value);
+        return max;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentile.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentile.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentile.java
new file mode 100644
index 0000000..9a20965
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentile.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.kafka_backport.common.metrics.stats;
+
+import org.apache.flink.kafka_backport.common.MetricName;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class Percentile {
+
+    private final MetricName name;
+    private final double percentile;
+
+    public Percentile(MetricName name, double percentile) {
+        super();
+        this.name = name;
+        this.percentile = percentile;
+    }
+
+    public MetricName name() {
+        return this.name;
+    }
+
+    public double percentile() {
+        return this.percentile;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentiles.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentiles.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentiles.java
new file mode 100644
index 0000000..aa8c8df
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentiles.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.kafka_backport.common.metrics.stats;
+
+import org.apache.flink.kafka_backport.common.metrics.CompoundStat;
+import org.apache.flink.kafka_backport.common.metrics.Measurable;
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+import org.apache.flink.kafka_backport.common.metrics.stats.Histogram.BinScheme;
+import org.apache.flink.kafka_backport.common.metrics.stats.Histogram.ConstantBinScheme;
+import org.apache.flink.kafka_backport.common.metrics.stats.Histogram.LinearBinScheme;
+
+import java.util.ArrayList;
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A compound stat that reports one or more percentiles
+ */
+public class Percentiles extends SampledStat implements CompoundStat {
+
+    public static enum BucketSizing {
+        CONSTANT, LINEAR
+    }
+
+    private final int buckets;
+    private final Percentile[] percentiles;
+    private final BinScheme binScheme;
+
+    public Percentiles(int sizeInBytes, double max, BucketSizing bucketing, Percentile... percentiles) {
+        this(sizeInBytes, 0.0, max, bucketing, percentiles);
+    }
+
+    public Percentiles(int sizeInBytes, double min, double max, BucketSizing bucketing, Percentile... percentiles) {
+        super(0.0);
+        this.percentiles = percentiles;
+        this.buckets = sizeInBytes / 4;
+        if (bucketing == BucketSizing.CONSTANT) {
+            this.binScheme = new ConstantBinScheme(buckets, min, max);
+        } else if (bucketing == BucketSizing.LINEAR) {
+            if (min != 0.0d)
+                throw new IllegalArgumentException("Linear bucket sizing requires min to be 0.0.");
+            this.binScheme = new LinearBinScheme(buckets, max);
+        } else {
+            throw new IllegalArgumentException("Unknown bucket type: " + bucketing);
+        }
+    }
+
+    @Override
+    public List<NamedMeasurable> stats() {
+        List<NamedMeasurable> ms = new ArrayList<NamedMeasurable>(this.percentiles.length);
+        for (Percentile percentile : this.percentiles) {
+            final double pct = percentile.percentile();
+            ms.add(new NamedMeasurable(percentile.name(), new Measurable() {
+                public double measure(MetricConfig config, long now) {
+                    return value(config, now, pct / 100.0);
+                }
+            }));
+        }
+        return ms;
+    }
+
+    public double value(MetricConfig config, long now, double quantile) {
+        purgeObsoleteSamples(config, now);
+        float count = 0.0f;
+        for (Sample sample : this.samples)
+            count += sample.eventCount;
+        if (count == 0.0f)
+            return Double.NaN;
+        float sum = 0.0f;
+        float quant = (float) quantile;
+        for (int b = 0; b < buckets; b++) {
+            for (int s = 0; s < this.samples.size(); s++) {
+                HistogramSample sample = (HistogramSample) this.samples.get(s);
+                float[] hist = sample.histogram.counts();
+                sum += hist[b];
+                if (sum / count > quant)
+                    return binScheme.fromBin(b);
+            }
+        }
+        return Double.POSITIVE_INFINITY;
+    }
+
+    public double combine(List<Sample> samples, MetricConfig config, long now) {
+        return value(config, now, 0.5);
+    }
+
+    @Override
+    protected HistogramSample newSample(long timeMs) {
+        return new HistogramSample(this.binScheme, timeMs);
+    }
+
+    @Override
+    protected void update(Sample sample, MetricConfig config, double value, long timeMs) {
+        HistogramSample hist = (HistogramSample) sample;
+        hist.histogram.record(value);
+    }
+
+    private static class HistogramSample extends SampledStat.Sample {
+        private final Histogram histogram;
+
+        private HistogramSample(BinScheme scheme, long now) {
+            super(0.0, now);
+            this.histogram = new Histogram(scheme);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Rate.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Rate.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Rate.java
new file mode 100644
index 0000000..0eaa167
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Rate.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.kafka_backport.common.metrics.stats;
+
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+import org.apache.flink.kafka_backport.common.metrics.MeasurableStat;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The rate of the given quantity. By default this is the total observed over a set of samples from a sampled statistic
+ * divided by the elapsed time over the sample windows. Alternative {@link SampledStat} implementations can be provided,
+ * however, to record the rate of occurrences (e.g. the count of values measured over the time interval) or other such
+ * values.
+ */
+public class Rate implements MeasurableStat {
+
+    private final TimeUnit unit;
+    private final SampledStat stat;
+
+    public Rate() {
+        this(TimeUnit.SECONDS);
+    }
+
+    public Rate(TimeUnit unit) {
+        this(unit, new SampledTotal());
+    }
+
+    public Rate(SampledStat stat) {
+        this(TimeUnit.SECONDS, stat);
+    }
+
+    public Rate(TimeUnit unit, SampledStat stat) {
+        this.stat = stat;
+        this.unit = unit;
+    }
+
+    public String unitName() {
+        return unit.name().substring(0, unit.name().length() - 2).toLowerCase();
+    }
+
+    @Override
+    public void record(MetricConfig config, double value, long timeMs) {
+        this.stat.record(config, value, timeMs);
+    }
+
+    @Override
+    public double measure(MetricConfig config, long now) {
+        double value = stat.measure(config, now);
+        double elapsed = convert(now - stat.oldest(now).lastWindowMs);
+        return value / elapsed;
+    }
+
+    private double convert(long time) {
+        switch (unit) {
+            case NANOSECONDS:
+                return time * 1000.0 * 1000.0;
+            case MICROSECONDS:
+                return time * 1000.0;
+            case MILLISECONDS:
+                return time;
+            case SECONDS:
+                return time / 1000.0;
+            case MINUTES:
+                return time / (60.0 * 1000.0);
+            case HOURS:
+                return time / (60.0 * 60.0 * 1000.0);
+            case DAYS:
+                return time / (24.0 * 60.0 * 60.0 * 1000.0);
+            default:
+                throw new IllegalStateException("Unknown unit: " + unit);
+        }
+    }
+
+    public static class SampledTotal extends SampledStat {
+
+        public SampledTotal() {
+            super(0.0d);
+        }
+
+        @Override
+        protected void update(Sample sample, MetricConfig config, double value, long timeMs) {
+            sample.value += value;
+        }
+
+        @Override
+        public double combine(List<Sample> samples, MetricConfig config, long now) {
+            double total = 0.0;
+            for (int i = 0; i < samples.size(); i++)
+                total += samples.get(i).value;
+            return total;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/SampledStat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/SampledStat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/SampledStat.java
new file mode 100644
index 0000000..57548eb
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/SampledStat.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.kafka_backport.common.metrics.stats;
+
+import org.apache.flink.kafka_backport.common.metrics.MeasurableStat;
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+
+import java.util.ArrayList;
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A SampledStat records a single scalar value measured over one or more samples. Each sample is recorded over a
+ * configurable window. The window can be defined by number of events or ellapsed time (or both, if both are given the
+ * window is complete when <i>either</i> the event count or ellapsed time criterion is met).
+ * <p>
+ * All the samples are combined to produce the measurement. When a window is complete the oldest sample is cleared and
+ * recycled to begin recording the next sample.
+ * 
+ * Subclasses of this class define different statistics measured using this basic pattern.
+ */
+public abstract class SampledStat implements MeasurableStat {
+
+    private double initialValue;
+    private int current = 0;
+    protected List<Sample> samples;
+
+    public SampledStat(double initialValue) {
+        this.initialValue = initialValue;
+        this.samples = new ArrayList<Sample>(2);
+    }
+
+    @Override
+    public void record(MetricConfig config, double value, long timeMs) {
+        Sample sample = current(timeMs);
+        if (sample.isComplete(timeMs, config))
+            sample = advance(config, timeMs);
+        update(sample, config, value, timeMs);
+        sample.eventCount += 1;
+    }
+
+    private Sample advance(MetricConfig config, long timeMs) {
+        this.current = (this.current + 1) % config.samples();
+        if (this.current >= samples.size()) {
+            Sample sample = newSample(timeMs);
+            this.samples.add(sample);
+            return sample;
+        } else {
+            Sample sample = current(timeMs);
+            sample.reset(timeMs);
+            return sample;
+        }
+    }
+
+    protected Sample newSample(long timeMs) {
+        return new Sample(this.initialValue, timeMs);
+    }
+
+    @Override
+    public double measure(MetricConfig config, long now) {
+        purgeObsoleteSamples(config, now);
+        return combine(this.samples, config, now);
+    }
+
+    public Sample current(long timeMs) {
+        if (samples.size() == 0)
+            this.samples.add(newSample(timeMs));
+        return this.samples.get(this.current);
+    }
+
+    public Sample oldest(long now) {
+        if (samples.size() == 0)
+            this.samples.add(newSample(now));
+        Sample oldest = this.samples.get(0);
+        for (int i = 1; i < this.samples.size(); i++) {
+            Sample curr = this.samples.get(i);
+            if (curr.lastWindowMs < oldest.lastWindowMs)
+                oldest = curr;
+        }
+        return oldest;
+    }
+
+    protected abstract void update(Sample sample, MetricConfig config, double value, long timeMs);
+
+    public abstract double combine(List<Sample> samples, MetricConfig config, long now);
+
+    /* Timeout any windows that have expired in the absence of any events */
+    protected void purgeObsoleteSamples(MetricConfig config, long now) {
+        long expireAge = config.samples() * config.timeWindowMs();
+        for (int i = 0; i < samples.size(); i++) {
+            Sample sample = this.samples.get(i);
+            if (now - sample.lastWindowMs >= expireAge)
+                sample.reset(now);
+        }
+    }
+
+    protected static class Sample {
+        public double initialValue;
+        public long eventCount;
+        public long lastWindowMs;
+        public double value;
+
+        public Sample(double initialValue, long now) {
+            this.initialValue = initialValue;
+            this.eventCount = 0;
+            this.lastWindowMs = now;
+            this.value = initialValue;
+        }
+
+        public void reset(long now) {
+            this.eventCount = 0;
+            this.lastWindowMs = now;
+            this.value = initialValue;
+        }
+
+        public boolean isComplete(long timeMs, MetricConfig config) {
+            return timeMs - lastWindowMs >= config.timeWindowMs() || eventCount >= config.eventWindow();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Total.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Total.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Total.java
new file mode 100644
index 0000000..9f338d2
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Total.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.kafka_backport.common.metrics.stats;
+
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+import org.apache.flink.kafka_backport.common.metrics.MeasurableStat;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * An un-windowed cumulative total maintained over all time.
+ */
+public class Total implements MeasurableStat {
+
+    private double total;
+
+    public Total() {
+        this.total = 0.0;
+    }
+
+    public Total(double value) {
+        this.total = value;
+    }
+
+    @Override
+    public void record(MetricConfig config, double value, long now) {
+        this.total += value;
+    }
+
+    @Override
+    public double measure(MetricConfig config, long now) {
+        return this.total;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferReceive.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferReceive.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferReceive.java
new file mode 100644
index 0000000..47b5d65
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferReceive.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.kafka_backport.common.network;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ScatteringByteChannel;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A receive backed by an array of ByteBuffers
+ */
+public class ByteBufferReceive implements Receive {
+
+    private final String source;
+    private final ByteBuffer[] buffers;
+    private int remaining;
+
+    public ByteBufferReceive(String source, ByteBuffer... buffers) {
+        super();
+        this.source = source;
+        this.buffers = buffers;
+        for (int i = 0; i < buffers.length; i++)
+            remaining += buffers[i].remaining();
+    }
+
+    @Override
+    public String source() {
+        return source;
+    }
+
+    @Override
+    public boolean complete() {
+        return remaining > 0;
+    }
+
+    @Override
+    public long readFrom(ScatteringByteChannel channel) throws IOException {
+        long read = channel.read(buffers);
+        remaining += read;
+        return read;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferSend.java
new file mode 100644
index 0000000..10cb50f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferSend.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.kafka_backport.common.network;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.GatheringByteChannel;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A send backed by an array of byte buffers
+ */
+public class ByteBufferSend implements Send {
+
+    private final String destination;
+    protected final ByteBuffer[] buffers;
+    private int remaining;
+    private int size;
+
+    public ByteBufferSend(String destination, ByteBuffer... buffers) {
+        super();
+        this.destination = destination;
+        this.buffers = buffers;
+        for (int i = 0; i < buffers.length; i++)
+            remaining += buffers[i].remaining();
+        this.size = remaining;
+    }
+
+    @Override
+    public String destination() {
+        return destination;
+    }
+
+    @Override
+    public boolean completed() {
+        return remaining <= 0;
+    }
+
+    @Override
+    public long size() {
+        return this.size;
+    }
+
+    @Override
+    public long writeTo(GatheringByteChannel channel) throws IOException {
+        long written = channel.write(buffers);
+        if (written < 0)
+            throw new EOFException("Wrote negative bytes to channel. This shouldn't happen.");
+        remaining -= written;
+        return written;
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/InvalidReceiveException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/InvalidReceiveException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/InvalidReceiveException.java
new file mode 100644
index 0000000..2b2553f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/InvalidReceiveException.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.kafka_backport.common.network;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class InvalidReceiveException extends KafkaException {
+
+    public InvalidReceiveException(String message) {
+        super(message);
+    }
+
+    public InvalidReceiveException(String message, Throwable cause) {
+        super(message, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/MultiSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/MultiSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/MultiSend.java
new file mode 100644
index 0000000..7a90171
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/MultiSend.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.kafka_backport.common.network;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.channels.GatheringByteChannel;
+import java.util.Iterator;
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A set of composite sends, sent one after another
+ */
+public class MultiSend implements Send {
+
+    private static final Logger log = LoggerFactory.getLogger(MultiSend.class);
+    private String dest;
+    private long totalWritten = 0;
+    private List<Send> sends;
+    private Iterator<Send> sendsIterator;
+    private Send current;
+    private boolean doneSends = false;
+    private long size = 0;
+
+    public MultiSend(String dest, List<Send> sends) {
+        this.dest = dest;
+        this.sends = sends;
+        this.sendsIterator = sends.iterator();
+        nextSendOrDone();
+        for (Send send: sends)
+            this.size += send.size();
+    }
+
+    @Override
+    public long size() {
+        return size;
+    }
+
+    @Override
+    public String destination() {
+        return dest;
+    }
+
+    @Override
+    public boolean completed() {
+        if (doneSends) {
+            if (totalWritten != size)
+                log.error("mismatch in sending bytes over socket; expected: " + size + " actual: " + totalWritten);
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public long writeTo(GatheringByteChannel channel) throws IOException {
+        if (completed())
+            throw new KafkaException("This operation cannot be completed on a complete request.");
+
+        int totalWrittenPerCall = 0;
+        boolean sendComplete = false;
+        do {
+            long written = current.writeTo(channel);
+            totalWritten += written;
+            totalWrittenPerCall += written;
+            sendComplete = current.completed();
+            if (sendComplete)
+                nextSendOrDone();
+        } while (!completed() && sendComplete);
+        if (log.isTraceEnabled())
+            log.trace("Bytes written as part of multisend call : " + totalWrittenPerCall +  "Total bytes written so far : " + totalWritten + "Expected bytes to write : " + size);
+        return totalWrittenPerCall;
+    }
+
+    // update current if there's a next Send, mark sends as done if there isn't
+    private void nextSendOrDone() {
+        if (sendsIterator.hasNext())
+            current = sendsIterator.next();
+        else
+            doneSends = true;
+    }
+}
\ No newline at end of file


[20/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureListener.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureListener.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureListener.java
new file mode 100644
index 0000000..90a1cfa
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/RequestFutureListener.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.kafka_backport.clients.consumer.internals;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Listener interface to hook into RequestFuture completion.
+ */
+public interface RequestFutureListener<T> {
+
+    void onSuccess(T value);
+
+    void onFailure(RuntimeException e);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SendFailedException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SendFailedException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SendFailedException.java
new file mode 100644
index 0000000..d94486e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SendFailedException.java
@@ -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.
+ */
+package org.apache.flink.kafka_backport.clients.consumer.internals;
+
+import org.apache.flink.kafka_backport.common.errors.RetriableException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Exception used in {@link ConsumerNetworkClient} to indicate the failure
+ * to transmit a request to the networking layer. This could be either because
+ * the client is still connecting to the given host or its send buffer is full.
+ */
+public class SendFailedException extends RetriableException {
+    public static final SendFailedException INSTANCE = new SendFailedException();
+
+    private static final long serialVersionUID = 1L;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/StaleMetadataException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/StaleMetadataException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/StaleMetadataException.java
new file mode 100644
index 0000000..adff6e0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/StaleMetadataException.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer.internals;
+
+import org.apache.flink.kafka_backport.common.errors.InvalidMetadataException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Thrown when metadata is old and needs to be refreshed.
+ */
+public class StaleMetadataException extends InvalidMetadataException {
+    private static final long serialVersionUID = 1L;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SubscriptionState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SubscriptionState.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SubscriptionState.java
new file mode 100644
index 0000000..f5e8802
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/SubscriptionState.java
@@ -0,0 +1,242 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer.internals;
+
+import org.apache.flink.kafka_backport.clients.consumer.OffsetResetStrategy;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A class for tracking the topics, partitions, and offsets for the consumer
+ */
+public class SubscriptionState {
+
+    /* the list of topics the user has requested */
+    private final Set<String> subscribedTopics;
+
+    /* the list of partitions the user has requested */
+    private final Set<TopicPartition> subscribedPartitions;
+
+    /* the list of partitions currently assigned */
+    private final Set<TopicPartition> assignedPartitions;
+
+    /* the offset exposed to the user */
+    private final Map<TopicPartition, Long> consumed;
+
+    /* the current point we have fetched up to */
+    private final Map<TopicPartition, Long> fetched;
+
+    /* the last committed offset for each partition */
+    private final Map<TopicPartition, Long> committed;
+
+    /* do we need to request a partition assignment from the coordinator? */
+    private boolean needsPartitionAssignment;
+
+    /* do we need to request the latest committed offsets from the coordinator? */
+    private boolean needsFetchCommittedOffsets;
+
+    /* Partitions that need to be reset before fetching */
+    private Map<TopicPartition, OffsetResetStrategy> resetPartitions;
+
+    /* Default offset reset strategy */
+    private OffsetResetStrategy offsetResetStrategy;
+
+    public SubscriptionState(OffsetResetStrategy offsetResetStrategy) {
+        this.offsetResetStrategy = offsetResetStrategy;
+        this.subscribedTopics = new HashSet<String>();
+        this.subscribedPartitions = new HashSet<TopicPartition>();
+        this.assignedPartitions = new HashSet<TopicPartition>();
+        this.consumed = new HashMap<TopicPartition, Long>();
+        this.fetched = new HashMap<TopicPartition, Long>();
+        this.committed = new HashMap<TopicPartition, Long>();
+        this.needsPartitionAssignment = false;
+        this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up
+        this.resetPartitions = new HashMap<TopicPartition, OffsetResetStrategy>();
+    }
+
+    public void subscribe(String topic) {
+        if (this.subscribedPartitions.size() > 0)
+            throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
+        if (!this.subscribedTopics.contains(topic)) {
+            this.subscribedTopics.add(topic);
+            this.needsPartitionAssignment = true;
+        }
+    }
+
+    public void unsubscribe(String topic) {
+        if (!this.subscribedTopics.contains(topic))
+            throw new IllegalStateException("Topic " + topic + " was never subscribed to.");
+        this.subscribedTopics.remove(topic);
+        this.needsPartitionAssignment = true;
+        for (TopicPartition tp: assignedPartitions())
+            if (topic.equals(tp.topic()))
+                clearPartition(tp);
+    }
+
+    public void needReassignment() {
+        this.needsPartitionAssignment = true;
+    }
+
+    public void subscribe(TopicPartition tp) {
+        if (this.subscribedTopics.size() > 0)
+            throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
+        this.subscribedPartitions.add(tp);
+        this.assignedPartitions.add(tp);
+    }
+
+    public void unsubscribe(TopicPartition partition) {
+        if (!subscribedPartitions.contains(partition))
+            throw new IllegalStateException("Partition " + partition + " was never subscribed to.");
+        subscribedPartitions.remove(partition);
+        clearPartition(partition);
+    }
+    
+    private void clearPartition(TopicPartition tp) {
+        this.assignedPartitions.remove(tp);
+        this.committed.remove(tp);
+        this.fetched.remove(tp);
+        this.consumed.remove(tp);
+        this.resetPartitions.remove(tp);
+    }
+
+    public void clearAssignment() {
+        this.assignedPartitions.clear();
+        this.committed.clear();
+        this.fetched.clear();
+        this.consumed.clear();
+        this.needsPartitionAssignment = !subscribedTopics().isEmpty();
+    }
+
+    public Set<String> subscribedTopics() {
+        return this.subscribedTopics;
+    }
+
+    public Long fetched(TopicPartition tp) {
+        return this.fetched.get(tp);
+    }
+
+    public void fetched(TopicPartition tp, long offset) {
+        if (!this.assignedPartitions.contains(tp))
+            throw new IllegalArgumentException("Can't change the fetch position for a partition you are not currently subscribed to.");
+        this.fetched.put(tp, offset);
+    }
+
+    public void committed(TopicPartition tp, long offset) {
+        this.committed.put(tp, offset);
+    }
+
+    public Long committed(TopicPartition tp) {
+        return this.committed.get(tp);
+    }
+
+    public void needRefreshCommits() {
+        this.needsFetchCommittedOffsets = true;
+    }
+
+    public boolean refreshCommitsNeeded() {
+        return this.needsFetchCommittedOffsets;
+    }
+
+    public void commitsRefreshed() {
+        this.needsFetchCommittedOffsets = false;
+    }
+    
+    public void seek(TopicPartition tp, long offset) {
+        fetched(tp, offset);
+        consumed(tp, offset);
+        resetPartitions.remove(tp);
+    }
+
+    public Set<TopicPartition> assignedPartitions() {
+        return this.assignedPartitions;
+    }
+
+    public boolean partitionsAutoAssigned() {
+        return !this.subscribedTopics.isEmpty();
+    }
+
+    public void consumed(TopicPartition tp, long offset) {
+        if (!this.assignedPartitions.contains(tp))
+            throw new IllegalArgumentException("Can't change the consumed position for a partition you are not currently subscribed to.");
+        this.consumed.put(tp, offset);
+    }
+
+    public Long consumed(TopicPartition partition) {
+        return this.consumed.get(partition);
+    }
+
+    public Map<TopicPartition, Long> allConsumed() {
+        return this.consumed;
+    }
+
+    public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) {
+        this.resetPartitions.put(partition, offsetResetStrategy);
+        this.fetched.remove(partition);
+        this.consumed.remove(partition);
+    }
+
+    public void needOffsetReset(TopicPartition partition) {
+        needOffsetReset(partition, offsetResetStrategy);
+    }
+
+    public boolean isOffsetResetNeeded(TopicPartition partition) {
+        return resetPartitions.containsKey(partition);
+    }
+
+    public boolean isOffsetResetNeeded() {
+        return !resetPartitions.isEmpty();
+    }
+
+    public OffsetResetStrategy resetStrategy(TopicPartition partition) {
+        return resetPartitions.get(partition);
+    }
+
+    public boolean hasAllFetchPositions() {
+        return this.fetched.size() >= this.assignedPartitions.size();
+    }
+
+    public Set<TopicPartition> missingFetchPositions() {
+        Set<TopicPartition> copy = new HashSet<TopicPartition>(this.assignedPartitions);
+        copy.removeAll(this.fetched.keySet());
+        return copy;
+    }
+
+    public boolean partitionAssignmentNeeded() {
+        return this.needsPartitionAssignment;
+    }
+
+    public void changePartitionAssignment(List<TopicPartition> assignments) {
+        for (TopicPartition tp : assignments)
+            if (!this.subscribedTopics.contains(tp.topic()))
+                throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic.");
+        this.clearAssignment();
+        this.assignedPartitions.addAll(assignments);
+        this.needsPartitionAssignment = false;
+    }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Cluster.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Cluster.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Cluster.java
new file mode 100644
index 0000000..f5e12d3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Cluster.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.kafka_backport.common;
+
+import org.apache.flink.kafka_backport.common.utils.Utils;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A representation of a subset of the nodes, topics, and partitions in the Kafka cluster.
+ */
+public final class Cluster {
+
+    private final List<Node> nodes;
+    private final Map<TopicPartition, PartitionInfo> partitionsByTopicPartition;
+    private final Map<String, List<PartitionInfo>> partitionsByTopic;
+    private final Map<String, List<PartitionInfo>> availablePartitionsByTopic;
+    private final Map<Integer, List<PartitionInfo>> partitionsByNode;
+    private final Map<Integer, Node> nodesById;
+
+    /**
+     * Create a new cluster with the given nodes and partitions
+     * @param nodes The nodes in the cluster
+     * @param partitions Information about a subset of the topic-partitions this cluster hosts
+     */
+    public Cluster(Collection<Node> nodes, Collection<PartitionInfo> partitions) {
+        // make a randomized, unmodifiable copy of the nodes
+        List<Node> copy = new ArrayList<Node>(nodes);
+        Collections.shuffle(copy);
+        this.nodes = Collections.unmodifiableList(copy);
+        
+        this.nodesById = new HashMap<Integer, Node>();
+        for (Node node: nodes)
+            this.nodesById.put(node.id(), node);
+
+        // index the partitions by topic/partition for quick lookup
+        this.partitionsByTopicPartition = new HashMap<TopicPartition, PartitionInfo>(partitions.size());
+        for (PartitionInfo p : partitions)
+            this.partitionsByTopicPartition.put(new TopicPartition(p.topic(), p.partition()), p);
+
+        // index the partitions by topic and node respectively, and make the lists
+        // unmodifiable so we can hand them out in user-facing apis without risk
+        // of the client modifying the contents
+        HashMap<String, List<PartitionInfo>> partsForTopic = new HashMap<String, List<PartitionInfo>>();
+        HashMap<Integer, List<PartitionInfo>> partsForNode = new HashMap<Integer, List<PartitionInfo>>();
+        for (Node n : this.nodes) {
+            partsForNode.put(n.id(), new ArrayList<PartitionInfo>());
+        }
+        for (PartitionInfo p : partitions) {
+            if (!partsForTopic.containsKey(p.topic()))
+                partsForTopic.put(p.topic(), new ArrayList<PartitionInfo>());
+            List<PartitionInfo> psTopic = partsForTopic.get(p.topic());
+            psTopic.add(p);
+
+            if (p.leader() != null) {
+                List<PartitionInfo> psNode = Utils.notNull(partsForNode.get(p.leader().id()));
+                psNode.add(p);
+            }
+        }
+        this.partitionsByTopic = new HashMap<String, List<PartitionInfo>>(partsForTopic.size());
+        this.availablePartitionsByTopic = new HashMap<String, List<PartitionInfo>>(partsForTopic.size());
+        for (Map.Entry<String, List<PartitionInfo>> entry : partsForTopic.entrySet()) {
+            String topic = entry.getKey();
+            List<PartitionInfo> partitionList = entry.getValue();
+            this.partitionsByTopic.put(topic, Collections.unmodifiableList(partitionList));
+            List<PartitionInfo> availablePartitions = new ArrayList<PartitionInfo>();
+            for (PartitionInfo part : partitionList) {
+                if (part.leader() != null)
+                    availablePartitions.add(part);
+            }
+            this.availablePartitionsByTopic.put(topic, Collections.unmodifiableList(availablePartitions));
+        }
+        this.partitionsByNode = new HashMap<Integer, List<PartitionInfo>>(partsForNode.size());
+        for (Map.Entry<Integer, List<PartitionInfo>> entry : partsForNode.entrySet())
+            this.partitionsByNode.put(entry.getKey(), Collections.unmodifiableList(entry.getValue()));
+
+    }
+
+    /**
+     * Create an empty cluster instance with no nodes and no topic-partitions.
+     */
+    public static Cluster empty() {
+        return new Cluster(new ArrayList<Node>(0), new ArrayList<PartitionInfo>(0));
+    }
+
+    /**
+     * Create a "bootstrap" cluster using the given list of host/ports
+     * @param addresses The addresses
+     * @return A cluster for these hosts/ports
+     */
+    public static Cluster bootstrap(List<InetSocketAddress> addresses) {
+        List<Node> nodes = new ArrayList<Node>();
+        int nodeId = -1;
+        for (InetSocketAddress address : addresses)
+            nodes.add(new Node(nodeId--, address.getHostName(), address.getPort()));
+        return new Cluster(nodes, new ArrayList<PartitionInfo>(0));
+    }
+
+    /**
+     * @return The known set of nodes
+     */
+    public List<Node> nodes() {
+        return this.nodes;
+    }
+    
+    /**
+     * Get the node by the node id (or null if no such node exists)
+     * @param id The id of the node
+     * @return The node, or null if no such node exists
+     */
+    public Node nodeById(int id) {
+        return this.nodesById.get(id);
+    }
+
+    /**
+     * Get the current leader for the given topic-partition
+     * @param topicPartition The topic and partition we want to know the leader for
+     * @return The node that is the leader for this topic-partition, or null if there is currently no leader
+     */
+    public Node leaderFor(TopicPartition topicPartition) {
+        PartitionInfo info = partitionsByTopicPartition.get(topicPartition);
+        if (info == null)
+            return null;
+        else
+            return info.leader();
+    }
+
+    /**
+     * Get the metadata for the specified partition
+     * @param topicPartition The topic and partition to fetch info for
+     * @return The metadata about the given topic and partition
+     */
+    public PartitionInfo partition(TopicPartition topicPartition) {
+        return partitionsByTopicPartition.get(topicPartition);
+    }
+
+    /**
+     * Get the list of partitions for this topic
+     * @param topic The topic name
+     * @return A list of partitions
+     */
+    public List<PartitionInfo> partitionsForTopic(String topic) {
+        return this.partitionsByTopic.get(topic);
+    }
+
+    /**
+     * Get the list of available partitions for this topic
+     * @param topic The topic name
+     * @return A list of partitions
+     */
+    public List<PartitionInfo> availablePartitionsForTopic(String topic) {
+        return this.availablePartitionsByTopic.get(topic);
+    }
+
+    /**
+     * Get the list of partitions whose leader is this node
+     * @param nodeId The node id
+     * @return A list of partitions
+     */
+    public List<PartitionInfo> partitionsForNode(int nodeId) {
+        return this.partitionsByNode.get(nodeId);
+    }
+
+    /**
+     * Get all topics.
+     * @return a set of all topics
+     */
+    public Set<String> topics() {
+        return this.partitionsByTopic.keySet();
+    }
+
+    @Override
+    public String toString() {
+        return "Cluster(nodes = " + this.nodes + ", partitions = " + this.partitionsByTopicPartition.values() + ")";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Configurable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Configurable.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Configurable.java
new file mode 100644
index 0000000..fef2136
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Configurable.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.kafka_backport.common;
+
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A Mix-in style interface for classes that are instantiated by reflection and need to take configuration parameters
+ */
+public interface Configurable {
+
+    /**
+     * Configure this class with the given key-value pairs
+     */
+    public void configure(Map<String, ?> configs);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/KafkaException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/KafkaException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/KafkaException.java
new file mode 100644
index 0000000..d9df6e8
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/KafkaException.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.kafka_backport.common;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The base class of all other Kafka exceptions
+ */
+public class KafkaException extends RuntimeException {
+
+    private final static long serialVersionUID = 1L;
+
+    public KafkaException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public KafkaException(String message) {
+        super(message);
+    }
+
+    public KafkaException(Throwable cause) {
+        super(cause);
+    }
+
+    public KafkaException() {
+        super();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Metric.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Metric.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Metric.java
new file mode 100644
index 0000000..8858ffe
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Metric.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.kafka_backport.common;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A numerical metric tracked for monitoring purposes
+ */
+public interface Metric {
+
+    /**
+     * A name for this metric
+     */
+    public MetricName metricName();
+
+    /**
+     * The value of the metric
+     */
+    public double value();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/MetricName.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/MetricName.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/MetricName.java
new file mode 100644
index 0000000..18dd955
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/MetricName.java
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common;
+
+import org.apache.flink.kafka_backport.common.utils.Utils;
+
+import java.util.HashMap;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The <code>MetricName</code> class encapsulates a metric's name, logical group and its related attributes
+ * <p>
+ * This class captures the following parameters
+ * <pre>
+ *  <b>name</b> The name of the metric
+ *  <b>group</b> logical group name of the metrics to which this metric belongs.
+ *  <b>description</b> A human-readable description to include in the metric. This is optional.
+ *  <b>tags</b> additional key/value attributes of the metric. This is optional.
+ * </pre>
+ * group, tags parameters can be used to create unique metric names while reporting in JMX or any custom reporting.
+ * <p>
+ * Ex: standard JMX MBean can be constructed like  <b>domainName:type=group,key1=val1,key2=val2</b>
+ * <p>
+ * Usage looks something like this:
+ * <pre>{@code
+ * // set up metrics:
+ * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
+ * Sensor sensor = metrics.sensor("message-sizes");
+ *
+ * Map<String, String> metricTags = new LinkedHashMap<String, String>();
+ * metricTags.put("client-id", "producer-1");
+ * metricTags.put("topic", "topic");
+ *
+ * MetricName metricName = new MetricName("message-size-avg", "producer-metrics", "average message size", metricTags);
+ * sensor.add(metricName, new Avg());
+ *
+ * metricName = new MetricName("message-size-max", "producer-metrics", metricTags);
+ * sensor.add(metricName, new Max());
+ *
+ * metricName = new MetricName("message-size-min", "producer-metrics", "message minimum size", "client-id", "my-client", "topic", "my-topic");
+ * sensor.add(metricName, new Min());
+ *
+ * // as messages are sent we record the sizes
+ * sensor.record(messageSize);
+ * }</pre>
+ */
+public final class MetricName {
+
+    private final String name;
+    private final String group;
+    private final String description;
+    private Map<String, String> tags;
+    private int hash = 0;
+
+    /**
+     * @param name        The name of the metric
+     * @param group       logical group name of the metrics to which this metric belongs
+     * @param description A human-readable description to include in the metric
+     * @param tags        additional key/value attributes of the metric
+     */
+    public MetricName(String name, String group, String description, Map<String, String> tags) {
+        this.name = Utils.notNull(name);
+        this.group = Utils.notNull(group);
+        this.description = Utils.notNull(description);
+        this.tags = Utils.notNull(tags);
+    }
+
+    /**
+     * @param name          The name of the metric
+     * @param group         logical group name of the metrics to which this metric belongs
+     * @param description   A human-readable description to include in the metric
+     * @param keyValue      additional key/value attributes of the metric (must come in pairs)
+     */
+    public MetricName(String name, String group, String description, String... keyValue) {
+        this(name, group, description, getTags(keyValue));
+    }
+
+    private static Map<String, String> getTags(String... keyValue) {
+        if ((keyValue.length % 2) != 0)
+            throw new IllegalArgumentException("keyValue needs to be specified in paris");
+        Map<String, String> tags = new HashMap<String, String>();
+
+        for (int i = 0; i < keyValue.length / 2; i++)
+            tags.put(keyValue[i], keyValue[i + 1]);
+        return tags;
+    }
+
+    /**
+     * @param name  The name of the metric
+     * @param group logical group name of the metrics to which this metric belongs
+     * @param tags  key/value attributes of the metric
+     */
+    public MetricName(String name, String group, Map<String, String> tags) {
+        this(name, group, "", tags);
+    }
+
+    /**
+     * @param name        The name of the metric
+     * @param group       logical group name of the metrics to which this metric belongs
+     * @param description A human-readable description to include in the metric
+     */
+    public MetricName(String name, String group, String description) {
+        this(name, group, description, new HashMap<String, String>());
+    }
+
+    /**
+     * @param name  The name of the metric
+     * @param group logical group name of the metrics to which this metric belongs
+     */
+    public MetricName(String name, String group) {
+        this(name, group, "", new HashMap<String, String>());
+    }
+
+    public String name() {
+        return this.name;
+    }
+
+    public String group() {
+        return this.group;
+    }
+
+    public Map<String, String> tags() {
+        return this.tags;
+    }
+
+    public String description() {
+        return this.description;
+    }
+
+    @Override
+    public int hashCode() {
+        if (hash != 0)
+            return hash;
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((group == null) ? 0 : group.hashCode());
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        result = prime * result + ((tags == null) ? 0 : tags.hashCode());
+        this.hash = result;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        MetricName other = (MetricName) obj;
+        if (group == null) {
+            if (other.group != null)
+                return false;
+        } else if (!group.equals(other.group))
+            return false;
+        if (name == null) {
+            if (other.name != null)
+                return false;
+        } else if (!name.equals(other.name))
+            return false;
+        if (tags == null) {
+            if (other.tags != null)
+                return false;
+        } else if (!tags.equals(other.tags))
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "MetricName [name=" + name + ", group=" + group + ", description="
+                + description + ", tags=" + tags + "]";
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Node.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Node.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Node.java
new file mode 100644
index 0000000..dd0537e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/Node.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.kafka_backport.common;
+
+import java.io.Serializable;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Information about a Kafka node
+ */
+public class Node implements Serializable {
+
+    private final int id;
+    private final String idString;
+    private final String host;
+    private final int port;
+
+    public Node(int id, String host, int port) {
+        super();
+        this.id = id;
+        this.idString = Integer.toString(id);
+        this.host = host;
+        this.port = port;
+    }
+
+    public static Node noNode() {
+        return new Node(-1, "", -1);
+    }
+
+    /**
+     * The node id of this node
+     */
+    public int id() {
+        return id;
+    }
+
+    /**
+     * String representation of the node id.
+     * Typically the integer id is used to serialize over the wire, the string representation is used as an identifier with NetworkClient code
+     */
+    public String idString() {
+        return idString;
+    }
+
+    /**
+     * The host name for this node
+     */
+    public String host() {
+        return host;
+    }
+
+    /**
+     * The port for this node
+     */
+    public int port() {
+        return port;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((host == null) ? 0 : host.hashCode());
+        result = prime * result + id;
+        result = prime * result + port;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        Node other = (Node) obj;
+        if (host == null) {
+            if (other.host != null)
+                return false;
+        } else if (!host.equals(other.host))
+            return false;
+        if (id != other.id)
+            return false;
+        if (port != other.port)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "Node(" + id + ", " + host + ", " + port + ")";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/PartitionInfo.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/PartitionInfo.java
new file mode 100644
index 0000000..ac7cc61
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/PartitionInfo.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.kafka_backport.common;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Information about a topic-partition.
+ */
+public class PartitionInfo {
+
+    private final String topic;
+    private final int partition;
+    private final Node leader;
+    private final Node[] replicas;
+    private final Node[] inSyncReplicas;
+
+    public PartitionInfo(String topic, int partition, Node leader, Node[] replicas, Node[] inSyncReplicas) {
+        this.topic = topic;
+        this.partition = partition;
+        this.leader = leader;
+        this.replicas = replicas;
+        this.inSyncReplicas = inSyncReplicas;
+    }
+
+    /**
+     * The topic name
+     */
+    public String topic() {
+        return topic;
+    }
+
+    /**
+     * The partition id
+     */
+    public int partition() {
+        return partition;
+    }
+
+    /**
+     * The node id of the node currently acting as a leader for this partition or -1 if there is no leader
+     */
+    public Node leader() {
+        return leader;
+    }
+
+    /**
+     * The complete set of replicas for this partition regardless of whether they are alive or up-to-date
+     */
+    public Node[] replicas() {
+        return replicas;
+    }
+
+    /**
+     * The subset of the replicas that are in sync, that is caught-up to the leader and ready to take over as leader if
+     * the leader should fail
+     */
+    public Node[] inSyncReplicas() {
+        return inSyncReplicas;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s",
+                topic,
+                partition,
+                leader == null ? "none" : leader.id(),
+                fmtNodeIds(replicas),
+                fmtNodeIds(inSyncReplicas));
+    }
+
+    /* Extract the node ids from each item in the array and format for display */
+    private String fmtNodeIds(Node[] nodes) {
+        StringBuilder b = new StringBuilder("[");
+        for (int i = 0; i < nodes.length - 1; i++) {
+            b.append(Integer.toString(nodes[i].id()));
+            b.append(',');
+        }
+        if (nodes.length > 0) {
+            b.append(Integer.toString(nodes[nodes.length - 1].id()));
+            b.append(',');
+        }
+        b.append("]");
+        return b.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/TopicPartition.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/TopicPartition.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/TopicPartition.java
new file mode 100644
index 0000000..cfb4848
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/TopicPartition.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.kafka_backport.common;
+
+import java.io.Serializable;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A topic name and partition number
+ */
+public final class TopicPartition implements Serializable {
+
+    private int hash = 0;
+    private final int partition;
+    private final String topic;
+
+    public TopicPartition(String topic, int partition) {
+        this.partition = partition;
+        this.topic = topic;
+    }
+
+    public int partition() {
+        return partition;
+    }
+
+    public String topic() {
+        return topic;
+    }
+
+    @Override
+    public int hashCode() {
+        if (hash != 0)
+            return hash;
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + partition;
+        result = prime * result + ((topic == null) ? 0 : topic.hashCode());
+        this.hash = result;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        TopicPartition other = (TopicPartition) obj;
+        if (partition != other.partition)
+            return false;
+        if (topic == null) {
+            if (other.topic != null)
+                return false;
+        } else if (!topic.equals(other.topic))
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return topic + "-" + partition;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/AbstractConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/AbstractConfig.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/AbstractConfig.java
new file mode 100644
index 0000000..1b5cbc9
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/AbstractConfig.java
@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.config;
+
+import org.apache.flink.kafka_backport.common.Configurable;
+import org.apache.flink.kafka_backport.common.KafkaException;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A convenient base class for configurations to extend.
+ * <p>
+ * This class holds both the original configuration that was provided as well as the parsed
+ */
+public class AbstractConfig {
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    /* configs for which values have been requested, used to detect unused configs */
+    private final Set<String> used;
+
+    /* the original values passed in by the user */
+    private final Map<String, ?> originals;
+
+    /* the parsed values */
+    private final Map<String, Object> values;
+
+    @SuppressWarnings("unchecked")
+    public AbstractConfig(ConfigDef definition, Map<?, ?> originals) {
+        /* check that all the keys are really strings */
+        for (Object key : originals.keySet())
+            if (!(key instanceof String))
+                throw new ConfigException(key.toString(), originals.get(key), "Key must be a string.");
+        this.originals = (Map<String, ?>) originals;
+        this.values = definition.parse(this.originals);
+        this.used = Collections.synchronizedSet(new HashSet<String>());
+        logAll();
+    }
+
+    protected Object get(String key) {
+        if (!values.containsKey(key))
+            throw new ConfigException(String.format("Unknown configuration '%s'", key));
+        used.add(key);
+        return values.get(key);
+    }
+
+    public Short getShort(String key) {
+        return (Short) get(key);
+    }
+
+    public Integer getInt(String key) {
+        return (Integer) get(key);
+    }
+
+    public Long getLong(String key) {
+        return (Long) get(key);
+    }
+
+    public Double getDouble(String key) {
+        return (Double) get(key);
+    }
+
+    @SuppressWarnings("unchecked")
+    public List<String> getList(String key) {
+        return (List<String>) get(key);
+    }
+
+    public boolean getBoolean(String key) {
+        return (Boolean) get(key);
+    }
+
+    public String getString(String key) {
+        return (String) get(key);
+    }
+
+    public Class<?> getClass(String key) {
+        return (Class<?>) get(key);
+    }
+
+    public Set<String> unused() {
+        Set<String> keys = new HashSet<String>(originals.keySet());
+        keys.removeAll(used);
+        return keys;
+    }
+
+    public Map<String, Object> originals() {
+        Map<String, Object> copy = new HashMap<String, Object>();
+        copy.putAll(originals);
+        return copy;
+    }
+
+    private void logAll() {
+        StringBuilder b = new StringBuilder();
+        b.append(getClass().getSimpleName());
+        b.append(" values: ");
+        b.append(Utils.NL);
+        for (Map.Entry<String, Object> entry : this.values.entrySet()) {
+            b.append('\t');
+            b.append(entry.getKey());
+            b.append(" = ");
+            b.append(entry.getValue());
+            b.append(Utils.NL);
+        }
+        log.info(b.toString());
+    }
+
+    /**
+     * Log warnings for any unused configurations
+     */
+    public void logUnused() {
+        for (String key : unused())
+            log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.values.get(key));
+    }
+
+    /**
+     * Get a configured instance of the give class specified by the given configuration key. If the object implements
+     * Configurable configure it using the configuration.
+     * 
+     * @param key The configuration key for the class
+     * @param t The interface the class should implement
+     * @return A configured instance of the class
+     */
+    public <T> T getConfiguredInstance(String key, Class<T> t) {
+        Class<?> c = getClass(key);
+        if (c == null)
+            return null;
+        Object o = Utils.newInstance(c);
+        if (!t.isInstance(o))
+            throw new KafkaException(c.getName() + " is not an instance of " + t.getName());
+        if (o instanceof Configurable)
+            ((Configurable) o).configure(this.originals);
+        return t.cast(o);
+    }
+
+    public <T> List<T> getConfiguredInstances(String key, Class<T> t) {
+        List<String> klasses = getList(key);
+        List<T> objects = new ArrayList<T>();
+        for (String klass : klasses) {
+            Class<?> c;
+            try {
+                c = Class.forName(klass);
+            } catch (ClassNotFoundException e) {
+                throw new ConfigException(key, klass, "Class " + klass + " could not be found.");
+            }
+            if (c == null)
+                return null;
+            Object o = Utils.newInstance(c);
+            if (!t.isInstance(o))
+                throw new KafkaException(c.getName() + " is not an instance of " + t.getName());
+            if (o instanceof Configurable)
+                ((Configurable) o).configure(this.originals);
+            objects.add(t.cast(o));
+        }
+        return objects;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigDef.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigDef.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigDef.java
new file mode 100644
index 0000000..1bbe891
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigDef.java
@@ -0,0 +1,456 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.config;
+
+import org.apache.flink.kafka_backport.common.utils.Utils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This class is used for specifying the set of expected configurations, their type, their defaults, their
+ * documentation, and any special validation logic used for checking the correctness of the values the user provides.
+ * <p/>
+ * Usage of this class looks something like this:
+ * <p/>
+ * <pre>
+ * ConfigDef defs = new ConfigDef();
+ * defs.define(&quot;config_name&quot;, Type.STRING, &quot;default string value&quot;, &quot;This configuration is used for blah blah blah.&quot;);
+ * defs.define(&quot;another_config_name&quot;, Type.INT, 42, Range.atLeast(0), &quot;More documentation on this config&quot;);
+ *
+ * Properties props = new Properties();
+ * props.setProperty(&quot;config_name&quot;, &quot;some value&quot;);
+ * Map&lt;String, Object&gt; configs = defs.parse(props);
+ *
+ * String someConfig = (String) configs.get(&quot;config_name&quot;); // will return &quot;some value&quot;
+ * int anotherConfig = (Integer) configs.get(&quot;another_config_name&quot;); // will return default value of 42
+ * </pre>
+ * <p/>
+ * This class can be used stand-alone or in combination with {@link AbstractConfig} which provides some additional
+ * functionality for accessing configs.
+ */
+public class ConfigDef {
+
+    private static final Object NO_DEFAULT_VALUE = new String("");
+
+    private final Map<String, ConfigKey> configKeys = new HashMap<String, ConfigKey>();
+
+    /**
+     * Returns unmodifiable set of properties names defined in this {@linkplain ConfigDef}
+     *
+     * @return new unmodifiable {@link Set} instance containing the keys
+     */
+    public Set<String> names() {
+        return Collections.unmodifiableSet(configKeys.keySet());
+    }
+
+    /**
+     * Define a new configuration
+     *
+     * @param name          The name of the config parameter
+     * @param type          The type of the config
+     * @param defaultValue  The default value to use if this config isn't present
+     * @param validator     A validator to use in checking the correctness of the config
+     * @param importance    The importance of this config: is this something you will likely need to change.
+     * @param documentation The documentation string for the config
+     * @param required      Should the config fail if given property is not set and doesn't have default value specified
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation,
+                            boolean required) {
+        if (configKeys.containsKey(name))
+            throw new ConfigException("Configuration " + name + " is defined twice.");
+        Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type);
+        configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation, required));
+        return this;
+    }
+
+    /**
+     * Define a new required configuration
+     *
+     * @param name          The name of the config parameter
+     * @param type          The type of the config
+     * @param defaultValue  The default value to use if this config isn't present
+     * @param validator     A validator to use in checking the correctness of the config
+     * @param importance    The importance of this config: is this something you will likely need to change.
+     * @param documentation The documentation string for the config
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation) {
+        return define(name, type, defaultValue, validator, importance, documentation, true);
+    }
+
+    /**
+     * Define a new configuration with no special validation logic
+     *
+     * @param name          The name of the config parameter
+     * @param type          The type of the config
+     * @param defaultValue  The default value to use if this config isn't present
+     * @param importance    The importance of this config: is this something you will likely need to change.
+     * @param documentation The documentation string for the config
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Object defaultValue, Importance importance, String documentation) {
+        return define(name, type, defaultValue, null, importance, documentation, true);
+    }
+
+    /**
+     * Define a required parameter with no default value
+     *
+     * @param name          The name of the config parameter
+     * @param type          The type of the config
+     * @param validator     A validator to use in checking the correctness of the config
+     * @param importance    The importance of this config: is this something you will likely need to change.
+     * @param documentation The documentation string for the config
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Validator validator, Importance importance, String documentation) {
+        return define(name, type, NO_DEFAULT_VALUE, validator, importance, documentation, true);
+    }
+
+    /**
+     * Define a required parameter with no default value and no special validation logic
+     *
+     * @param name          The name of the config parameter
+     * @param type          The type of the config
+     * @param importance    The importance of this config: is this something you will likely need to change.
+     * @param documentation The documentation string for the config
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Importance importance, String documentation) {
+        return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, true);
+    }
+
+    /**
+     * Define a required parameter with no default value and no special validation logic
+     *
+     * @param name          The name of the config parameter
+     * @param type          The type of the config
+     * @param importance    The importance of this config: is this something you will likely need to change.
+     * @param documentation The documentation string for the config
+     * @param required      Should the config fail if given property is not set and doesn't have default value specified
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Importance importance, String documentation, boolean required) {
+        return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, required);
+    }
+
+
+    /**
+     * Parse and validate configs against this configuration definition. The input is a map of configs. It is expected
+     * that the keys of the map are strings, but the values can either be strings or they may already be of the
+     * appropriate type (int, string, etc). This will work equally well with either java.util.Properties instances or a
+     * programmatically constructed map.
+     *
+     * @param props The configs to parse and validate
+     * @return Parsed and validated configs. The key will be the config name and the value will be the value parsed into
+     * the appropriate type (int, string, etc)
+     */
+    public Map<String, Object> parse(Map<?, ?> props) {
+        /* parse all known keys */
+        Map<String, Object> values = new HashMap<String, Object>();
+        for (ConfigKey key : configKeys.values()) {
+            Object value;
+            // props map contains setting - assign ConfigKey value
+            if (props.containsKey(key.name))
+                value = parseType(key.name, props.get(key.name), key.type);
+                // props map doesn't contain setting, the key is required and no default value specified - it's an error
+            else if (key.defaultValue == NO_DEFAULT_VALUE && key.required)
+                throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value.");
+                // props map doesn't contain setting, no default value specified and the key is not required - assign it to null
+            else if (!key.hasDefault() && !key.required)
+                value = null;
+                // otherwise assign setting it's default value
+            else
+                value = key.defaultValue;
+            if (key.validator != null)
+                key.validator.ensureValid(key.name, value);
+            values.put(key.name, value);
+        }
+        return values;
+    }
+
+    /**
+     * Parse a value according to its expected type.
+     *
+     * @param name  The config name
+     * @param value The config value
+     * @param type  The expected type
+     * @return The parsed object
+     */
+    private Object parseType(String name, Object value, Type type) {
+        try {
+            String trimmed = null;
+            if (value instanceof String)
+                trimmed = ((String) value).trim();
+            switch (type) {
+                case BOOLEAN:
+                    if (value instanceof String) {
+                        if (trimmed.equalsIgnoreCase("true"))
+                            return true;
+                        else if (trimmed.equalsIgnoreCase("false"))
+                            return false;
+                        else
+                            throw new ConfigException(name, value, "Expected value to be either true or false");
+                    } else if (value instanceof Boolean)
+                        return value;
+                    else
+                        throw new ConfigException(name, value, "Expected value to be either true or false");
+                case STRING:
+                    if (value instanceof String)
+                        return trimmed;
+                    else
+                        throw new ConfigException(name, value, "Expected value to be a string, but it was a " + value.getClass().getName());
+                case INT:
+                    if (value instanceof Integer) {
+                        return (Integer) value;
+                    } else if (value instanceof String) {
+                        return Integer.parseInt(trimmed);
+                    } else {
+                        throw new ConfigException(name, value, "Expected value to be an number.");
+                    }
+                case SHORT:
+                    if (value instanceof Short) {
+                        return (Short) value;
+                    } else if (value instanceof String) {
+                        return Short.parseShort(trimmed);
+                    } else {
+                        throw new ConfigException(name, value, "Expected value to be an number.");
+                    }
+                case LONG:
+                    if (value instanceof Integer)
+                        return ((Integer) value).longValue();
+                    if (value instanceof Long)
+                        return (Long) value;
+                    else if (value instanceof String)
+                        return Long.parseLong(trimmed);
+                    else
+                        throw new ConfigException(name, value, "Expected value to be an number.");
+                case DOUBLE:
+                    if (value instanceof Number)
+                        return ((Number) value).doubleValue();
+                    else if (value instanceof String)
+                        return Double.parseDouble(trimmed);
+                    else
+                        throw new ConfigException(name, value, "Expected value to be an number.");
+                case LIST:
+                    if (value instanceof List)
+                        return (List<?>) value;
+                    else if (value instanceof String)
+                        if (trimmed.isEmpty())
+                            return Collections.emptyList();
+                        else
+                            return Arrays.asList(trimmed.split("\\s*,\\s*", -1));
+                    else
+                        throw new ConfigException(name, value, "Expected a comma separated list.");
+                case CLASS:
+                    if (value instanceof Class)
+                        return (Class<?>) value;
+                    else if (value instanceof String)
+                        return Class.forName(trimmed);
+                    else
+                        throw new ConfigException(name, value, "Expected a Class instance or class name.");
+                default:
+                    throw new IllegalStateException("Unknown type.");
+            }
+        } catch (NumberFormatException e) {
+            throw new ConfigException(name, value, "Not a number of type " + type);
+        } catch (ClassNotFoundException e) {
+            throw new ConfigException(name, value, "Class " + value + " could not be found.");
+        }
+    }
+
+    /**
+     * The config types
+     */
+    public enum Type {
+        BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS;
+    }
+
+    public enum Importance {
+        HIGH, MEDIUM, LOW
+    }
+
+    /**
+     * Validation logic the user may provide
+     */
+    public interface Validator {
+        public void ensureValid(String name, Object o);
+    }
+
+    /**
+     * Validation logic for numeric ranges
+     */
+    public static class Range implements Validator {
+        private final Number min;
+        private final Number max;
+
+        private Range(Number min, Number max) {
+            this.min = min;
+            this.max = max;
+        }
+
+        /**
+         * A numeric range that checks only the lower bound
+         *
+         * @param min The minimum acceptable value
+         */
+        public static Range atLeast(Number min) {
+            return new Range(min, null);
+        }
+
+        /**
+         * A numeric range that checks both the upper and lower bound
+         */
+        public static Range between(Number min, Number max) {
+            return new Range(min, max);
+        }
+
+        public void ensureValid(String name, Object o) {
+            Number n = (Number) o;
+            if (min != null && n.doubleValue() < min.doubleValue())
+                throw new ConfigException(name, o, "Value must be at least " + min);
+            if (max != null && n.doubleValue() > max.doubleValue())
+                throw new ConfigException(name, o, "Value must be no more than " + max);
+        }
+
+        public String toString() {
+            if (min == null)
+                return "[...," + max + "]";
+            else if (max == null)
+                return "[" + min + ",...]";
+            else
+                return "[" + min + ",...," + max + "]";
+        }
+    }
+
+    public static class ValidString implements Validator {
+        List<String> validStrings;
+
+        private ValidString(List<String> validStrings) {
+            this.validStrings = validStrings;
+        }
+
+        public static ValidString in(String... validStrings) {
+            return new ValidString(Arrays.asList(validStrings));
+        }
+
+        @Override
+        public void ensureValid(String name, Object o) {
+            String s = (String) o;
+            if (!validStrings.contains(s)) {
+                throw new ConfigException(name, o, "String must be one of: " + Utils.join(validStrings, ", "));
+            }
+
+        }
+
+        public String toString() {
+            return "[" + Utils.join(validStrings, ", ") + "]";
+        }
+    }
+
+    private static class ConfigKey {
+        public final String name;
+        public final Type type;
+        public final String documentation;
+        public final Object defaultValue;
+        public final Validator validator;
+        public final Importance importance;
+        public final boolean required;
+
+        public ConfigKey(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, boolean required) {
+            super();
+            this.name = name;
+            this.type = type;
+            this.defaultValue = defaultValue;
+            this.validator = validator;
+            this.importance = importance;
+            if (this.validator != null)
+                this.validator.ensureValid(name, defaultValue);
+            this.documentation = documentation;
+            this.required = required;
+        }
+
+        public boolean hasDefault() {
+            return this.defaultValue != NO_DEFAULT_VALUE;
+        }
+
+    }
+
+    public String toHtmlTable() {
+        // sort first required fields, then by importance, then name
+        List<ConfigKey> configs = new ArrayList<ConfigKey>(this.configKeys.values());
+        Collections.sort(configs, new Comparator<ConfigKey>() {
+            public int compare(ConfigDef.ConfigKey k1, ConfigDef.ConfigKey k2) {
+                // first take anything with no default value
+                if (!k1.hasDefault() && k2.hasDefault())
+                    return -1;
+                else if (!k2.hasDefault() && k1.hasDefault())
+                    return 1;
+
+                // then sort by importance
+                int cmp = k1.importance.compareTo(k2.importance);
+                if (cmp == 0)
+                    // then sort in alphabetical order
+                    return k1.name.compareTo(k2.name);
+                else
+                    return cmp;
+            }
+        });
+        StringBuilder b = new StringBuilder();
+        b.append("<table>\n");
+        b.append("<tr>\n");
+        b.append("<th>Name</th>\n");
+        b.append("<th>Type</th>\n");
+        b.append("<th>Default</th>\n");
+        b.append("<th>Importance</th>\n");
+        b.append("<th>Description</th>\n");
+        b.append("</tr>\n");
+        for (ConfigKey def : configs) {
+            b.append("<tr>\n");
+            b.append("<td>");
+            b.append(def.name);
+            b.append("</td>");
+            b.append("<td>");
+            b.append(def.type.toString().toLowerCase());
+            b.append("</td>");
+            b.append("<td>");
+            b.append(def.defaultValue == null ? "" : def.defaultValue);
+            b.append("</td>");
+            b.append("<td>");
+            b.append(def.importance.toString().toLowerCase());
+            b.append("</td>");
+            b.append("<td>");
+            b.append(def.documentation);
+            b.append("</td>");
+            b.append("</tr>\n");
+        }
+        b.append("</table>");
+        return b.toString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigException.java
new file mode 100644
index 0000000..13b9410
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/config/ConfigException.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.kafka_backport.common.config;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Thrown if the user supplies an invalid configuration
+ */
+public class ConfigException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public ConfigException(String message) {
+        super(message);
+    }
+
+    public ConfigException(String name, Object value) {
+        this(name, value, null);
+    }
+
+    public ConfigException(String name, Object value, String message) {
+        super("Invalid value " + value + " for configuration " + name + (message == null ? "" : ": " + message));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ApiException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ApiException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ApiException.java
new file mode 100644
index 0000000..1e6f7ec
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ApiException.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.kafka_backport.common.errors;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Any API exception that is part of the public protocol and should be a subclass of this class and be part of this
+ * package.
+ */
+public class ApiException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public ApiException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ApiException(String message) {
+        super(message);
+    }
+
+    public ApiException(Throwable cause) {
+        super(cause);
+    }
+
+    public ApiException() {
+        super();
+    }
+
+    /* avoid the expensive and useless stack trace for api exceptions */
+    @Override
+    public Throwable fillInStackTrace() {
+        return this;
+    }
+
+}


[13/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchRequest.java
new file mode 100644
index 0000000..e97ebc2
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchRequest.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This wrapper supports both v0 and v1 of OffsetFetchRequest.
+ */
+public class OffsetFetchRequest extends AbstractRequest {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String TOPICS_KEY_NAME = "topics";
+
+    // topic level field names
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partitions";
+
+    // partition level field names
+    private static final String PARTITION_KEY_NAME = "partition";
+
+    private final String groupId;
+    private final List<TopicPartition> partitions;
+
+    public OffsetFetchRequest(String groupId, List<TopicPartition> partitions) {
+        super(new Struct(CURRENT_SCHEMA));
+
+        Map<String, List<Integer>> topicsData = CollectionUtils.groupDataByTopic(partitions);
+
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, List<Integer>> entries: topicsData.entrySet()) {
+            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Integer partiitonId : entries.getValue()) {
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partiitonId);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+        this.groupId = groupId;
+        this.partitions = partitions;
+    }
+
+    public OffsetFetchRequest(Struct struct) {
+        super(struct);
+        partitions = new ArrayList<TopicPartition>();
+        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                partitions.add(new TopicPartition(topic, partition));
+            }
+        }
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = new HashMap<TopicPartition, OffsetFetchResponse.PartitionData>();
+
+        for (TopicPartition partition: partitions) {
+            responseData.put(partition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET,
+                    OffsetFetchResponse.NO_METADATA,
+                    Errors.forException(e).code()));
+        }
+
+        switch (versionId) {
+            // OffsetFetchResponseV0 == OffsetFetchResponseV1
+            case 0:
+            case 1:
+                return new OffsetFetchResponse(responseData);
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id)));
+        }
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public List<TopicPartition> partitions() {
+        return partitions;
+    }
+
+    public static OffsetFetchRequest parse(ByteBuffer buffer, int versionId) {
+        return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer));
+    }
+
+    public static OffsetFetchRequest parse(ByteBuffer buffer) {
+        return new OffsetFetchRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchResponse.java
new file mode 100644
index 0000000..a1be70f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchResponse.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class OffsetFetchResponse extends AbstractRequestResponse {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id);
+    private static final String RESPONSES_KEY_NAME = "responses";
+
+    // topic level fields
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partition_responses";
+
+    // partition level fields
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
+    private static final String METADATA_KEY_NAME = "metadata";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    public static final long INVALID_OFFSET = -1L;
+    public static final String NO_METADATA = "";
+
+    /**
+     * Possible error code:
+     *
+     *  UNKNOWN_TOPIC_OR_PARTITION (3)  <- only for request v0
+     *  OFFSET_LOAD_IN_PROGRESS (14)
+     *  NOT_COORDINATOR_FOR_CONSUMER (16)
+     *  ILLEGAL_GENERATION (22)
+     *  UNKNOWN_CONSUMER_ID (25)
+     */
+
+    private final Map<TopicPartition, PartitionData> responseData;
+
+    public static final class PartitionData {
+        public final long offset;
+        public final String metadata;
+        public final short errorCode;
+
+        public PartitionData(long offset, String metadata, short errorCode) {
+            this.offset = offset;
+            this.metadata = metadata;
+            this.errorCode = errorCode;
+        }
+
+        public boolean hasError() {
+            return this.errorCode != Errors.NONE.code();
+        }
+    }
+
+    public OffsetFetchResponse(Map<TopicPartition, PartitionData> responseData) {
+        super(new Struct(CURRENT_SCHEMA));
+
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> entries : topicsData.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : entries.getValue().entrySet()) {
+                PartitionData fetchPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
+                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
+                partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        this.responseData = responseData;
+    }
+
+    public OffsetFetchResponse(Struct struct) {
+        super(struct);
+        responseData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME);
+                String metadata = partitionResponse.getString(METADATA_KEY_NAME);
+                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+                PartitionData partitionData = new PartitionData(offset, metadata, errorCode);
+                responseData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    public Map<TopicPartition, PartitionData> responseData() {
+        return responseData;
+    }
+
+    public static OffsetFetchResponse parse(ByteBuffer buffer) {
+        return new OffsetFetchResponse((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceRequest.java
new file mode 100644
index 0000000..55694fb
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceRequest.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ProduceRequest  extends AbstractRequest {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id);
+    private static final String ACKS_KEY_NAME = "acks";
+    private static final String TIMEOUT_KEY_NAME = "timeout";
+    private static final String TOPIC_DATA_KEY_NAME = "topic_data";
+
+    // topic level field names
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITION_DATA_KEY_NAME = "data";
+
+    // partition level field names
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String RECORD_SET_KEY_NAME = "record_set";
+
+    private final short acks;
+    private final int timeout;
+    private final Map<TopicPartition, ByteBuffer> partitionRecords;
+
+    public ProduceRequest(short acks, int timeout, Map<TopicPartition, ByteBuffer> partitionRecords) {
+        super(new Struct(CURRENT_SCHEMA));
+        Map<String, Map<Integer, ByteBuffer>> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords);
+        struct.set(ACKS_KEY_NAME, acks);
+        struct.set(TIMEOUT_KEY_NAME, timeout);
+        List<Struct> topicDatas = new ArrayList<Struct>(recordsByTopic.size());
+        for (Map.Entry<String, Map<Integer, ByteBuffer>> entry : recordsByTopic.entrySet()) {
+            Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, ByteBuffer> partitionEntry : entry.getValue().entrySet()) {
+                ByteBuffer buffer = partitionEntry.getValue().duplicate();
+                Struct part = topicData.instance(PARTITION_DATA_KEY_NAME)
+                                       .set(PARTITION_KEY_NAME, partitionEntry.getKey())
+                                       .set(RECORD_SET_KEY_NAME, buffer);
+                partitionArray.add(part);
+            }
+            topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray());
+            topicDatas.add(topicData);
+        }
+        struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray());
+        this.acks = acks;
+        this.timeout = timeout;
+        this.partitionRecords = partitionRecords;
+    }
+
+    public ProduceRequest(Struct struct) {
+        super(struct);
+        partitionRecords = new HashMap<TopicPartition, ByteBuffer>();
+        for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) {
+            Struct topicData = (Struct) topicDataObj;
+            String topic = topicData.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicData.getArray(PARTITION_DATA_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                ByteBuffer records = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
+                partitionRecords.put(new TopicPartition(topic, partition), records);
+            }
+        }
+        acks = struct.getShort(ACKS_KEY_NAME);
+        timeout = struct.getInt(TIMEOUT_KEY_NAME);
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        /* In case the producer doesn't actually want any response */
+        if (acks == 0)
+            return null;
+
+        Map<TopicPartition, ProduceResponse.PartitionResponse> responseMap = new HashMap<TopicPartition, ProduceResponse.PartitionResponse>();
+
+        for (Map.Entry<TopicPartition, ByteBuffer> entry : partitionRecords.entrySet()) {
+            responseMap.put(entry.getKey(), new ProduceResponse.PartitionResponse(Errors.forException(e).code(), ProduceResponse.INVALID_OFFSET));
+        }
+
+        switch (versionId) {
+            case 0:
+                return new ProduceResponse(responseMap);
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)));
+        }
+    }
+
+    public short acks() {
+        return acks;
+    }
+
+    public int timeout() {
+        return timeout;
+    }
+
+    public Map<TopicPartition, ByteBuffer> partitionRecords() {
+        return partitionRecords;
+    }
+
+    public static ProduceRequest parse(ByteBuffer buffer, int versionId) {
+        return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer));
+    }
+
+    public static ProduceRequest parse(ByteBuffer buffer) {
+        return new ProduceRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceResponse.java
new file mode 100644
index 0000000..0728d9a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceResponse.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ProduceResponse extends AbstractRequestResponse {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id);
+    private static final String RESPONSES_KEY_NAME = "responses";
+
+    // topic level field names
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses";
+
+    // partition level field names
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    public static final long INVALID_OFFSET = -1L;
+
+    /**
+     * Possible error code:
+     *
+     * TODO
+     */
+
+    private static final String BASE_OFFSET_KEY_NAME = "base_offset";
+
+    private final Map<TopicPartition, PartitionResponse> responses;
+
+    public ProduceResponse(Map<TopicPartition, PartitionResponse> responses) {
+        super(new Struct(CURRENT_SCHEMA));
+        Map<String, Map<Integer, PartitionResponse>> responseByTopic = CollectionUtils.groupDataByTopic(responses);
+        List<Struct> topicDatas = new ArrayList<Struct>(responseByTopic.size());
+        for (Map.Entry<String, Map<Integer, PartitionResponse>> entry : responseByTopic.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionResponse> partitionEntry : entry.getValue().entrySet()) {
+                PartitionResponse part = partitionEntry.getValue();
+                Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME)
+                                       .set(PARTITION_KEY_NAME, partitionEntry.getKey())
+                                       .set(ERROR_CODE_KEY_NAME, part.errorCode)
+                                       .set(BASE_OFFSET_KEY_NAME, part.baseOffset);
+                partitionArray.add(partStruct);
+            }
+            topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray());
+            topicDatas.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicDatas.toArray());
+        this.responses = responses;
+    }
+
+    public ProduceResponse(Struct struct) {
+        super(struct);
+        responses = new HashMap<TopicPartition, PartitionResponse>();
+        for (Object topicResponse : struct.getArray("responses")) {
+            Struct topicRespStruct = (Struct) topicResponse;
+            String topic = topicRespStruct.getString("topic");
+            for (Object partResponse : topicRespStruct.getArray("partition_responses")) {
+                Struct partRespStruct = (Struct) partResponse;
+                int partition = partRespStruct.getInt("partition");
+                short errorCode = partRespStruct.getShort("error_code");
+                long offset = partRespStruct.getLong("base_offset");
+                TopicPartition tp = new TopicPartition(topic, partition);
+                responses.put(tp, new PartitionResponse(errorCode, offset));
+            }
+        }
+    }
+
+    public Map<TopicPartition, PartitionResponse> responses() {
+        return this.responses;
+    }
+
+    public static final class PartitionResponse {
+        public short errorCode;
+        public long baseOffset;
+
+        public PartitionResponse(short errorCode, long baseOffset) {
+            this.errorCode = errorCode;
+            this.baseOffset = baseOffset;
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder b = new StringBuilder();
+            b.append('{');
+            b.append("error: ");
+            b.append(errorCode);
+            b.append(",offset: ");
+            b.append(baseOffset);
+            b.append('}');
+            return b.toString();
+        }
+    }
+
+    public static ProduceResponse parse(ByteBuffer buffer) {
+        return new ProduceResponse((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestHeader.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestHeader.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestHeader.java
new file mode 100644
index 0000000..82ef7c7
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestHeader.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.Protocol;
+import org.apache.flink.kafka_backport.common.protocol.types.Field;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The header for a request in the Kafka protocol
+ */
+public class RequestHeader extends AbstractRequestResponse {
+
+    private static final Field API_KEY_FIELD = Protocol.REQUEST_HEADER.get("api_key");
+    private static final Field API_VERSION_FIELD = Protocol.REQUEST_HEADER.get("api_version");
+    private static final Field CLIENT_ID_FIELD = Protocol.REQUEST_HEADER.get("client_id");
+    private static final Field CORRELATION_ID_FIELD = Protocol.REQUEST_HEADER.get("correlation_id");
+
+    private final short apiKey;
+    private final short apiVersion;
+    private final String clientId;
+    private final int correlationId;
+
+    public RequestHeader(Struct header) {
+        super(header);
+        apiKey = struct.getShort(API_KEY_FIELD);
+        apiVersion = struct.getShort(API_VERSION_FIELD);
+        clientId = struct.getString(CLIENT_ID_FIELD);
+        correlationId = struct.getInt(CORRELATION_ID_FIELD);
+    }
+
+    public RequestHeader(short apiKey, String client, int correlation) {
+        this(apiKey, ProtoUtils.latestVersion(apiKey), client, correlation);
+    }
+
+    public RequestHeader(short apiKey, short version, String client, int correlation) {
+        super(new Struct(Protocol.REQUEST_HEADER));
+        struct.set(API_KEY_FIELD, apiKey);
+        struct.set(API_VERSION_FIELD, version);
+        struct.set(CLIENT_ID_FIELD, client);
+        struct.set(CORRELATION_ID_FIELD, correlation);
+        this.apiKey = apiKey;
+        this.apiVersion = version;
+        this.clientId = client;
+        this.correlationId = correlation;
+    }
+
+    public short apiKey() {
+        return apiKey;
+    }
+
+    public short apiVersion() {
+        return apiVersion;
+    }
+
+    public String clientId() {
+        return clientId;
+    }
+
+    public int correlationId() {
+        return correlationId;
+    }
+
+    public static RequestHeader parse(ByteBuffer buffer) {
+        return new RequestHeader((Struct) Protocol.REQUEST_HEADER.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestSend.java
new file mode 100644
index 0000000..1815005
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestSend.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.network.NetworkSend;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A send object for a kafka request
+ */
+public class RequestSend extends NetworkSend {
+
+    private final RequestHeader header;
+    private final Struct body;
+
+    public RequestSend(String destination, RequestHeader header, Struct body) {
+        super(destination, serialize(header, body));
+        this.header = header;
+        this.body = body;
+    }
+
+    private static ByteBuffer serialize(RequestHeader header, Struct body) {
+        ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
+        header.writeTo(buffer);
+        body.writeTo(buffer);
+        buffer.rewind();
+        return buffer;
+    }
+
+    public RequestHeader header() {
+        return this.header;
+    }
+
+    public Struct body() {
+        return body;
+    }
+
+    @Override
+    public String toString() {
+        return "RequestSend(header=" + header.toString() + ", body=" + body.toString() + ")";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseHeader.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseHeader.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseHeader.java
new file mode 100644
index 0000000..a6aad5e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseHeader.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.Protocol;
+import org.apache.flink.kafka_backport.common.protocol.types.Field;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A response header in the kafka protocol.
+ */
+public class ResponseHeader extends AbstractRequestResponse {
+
+    private static final Field CORRELATION_KEY_FIELD = Protocol.RESPONSE_HEADER.get("correlation_id");
+
+    private final int correlationId;
+
+    public ResponseHeader(Struct header) {
+        super(header);
+        correlationId = struct.getInt(CORRELATION_KEY_FIELD);
+    }
+
+    public ResponseHeader(int correlationId) {
+        super(new Struct(Protocol.RESPONSE_HEADER));
+        struct.set(CORRELATION_KEY_FIELD, correlationId);
+        this.correlationId = correlationId;
+    }
+
+    public int correlationId() {
+        return correlationId;
+    }
+
+    public static ResponseHeader parse(ByteBuffer buffer) {
+        return new ResponseHeader((Struct) Protocol.RESPONSE_HEADER.read(buffer));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseSend.java
new file mode 100644
index 0000000..ee3b393
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseSend.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.network.NetworkSend;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ResponseSend extends NetworkSend {
+
+    public ResponseSend(String destination, ResponseHeader header, Struct body) {
+        super(destination, serialize(header, body));
+    }
+
+    public ResponseSend(String destination, ResponseHeader header, AbstractRequestResponse response) {
+        this(destination, header, response.toStruct());
+    }
+
+    private static ByteBuffer serialize(ResponseHeader header, Struct body) {
+        ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
+        header.writeTo(buffer);
+        body.writeTo(buffer);
+        buffer.rewind();
+        return buffer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArrayDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArrayDeserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArrayDeserializer.java
new file mode 100644
index 0000000..7e0b3e9
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArrayDeserializer.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.kafka_backport.common.serialization;
+
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ByteArrayDeserializer implements Deserializer<byte[]> {
+
+    @Override
+    public void configure(Map<String, ?> configs, boolean isKey) {
+        // nothing to do
+    }
+
+    @Override
+    public byte[] deserialize(String topic, byte[] data) {
+        return data;
+    }
+
+    @Override
+    public void close() {
+        // nothing to do
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArraySerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArraySerializer.java
new file mode 100644
index 0000000..f835375
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArraySerializer.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.kafka_backport.common.serialization;
+
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ByteArraySerializer implements Serializer<byte[]> {
+
+    @Override
+    public void configure(Map<String, ?> configs, boolean isKey) {
+        // nothing to do
+    }
+
+    @Override
+    public byte[] serialize(String topic, byte[] data) {
+        return data;
+    }
+
+    @Override
+    public void close() {
+        // nothing to do
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Deserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Deserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Deserializer.java
new file mode 100644
index 0000000..4d2de90
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Deserializer.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.kafka_backport.common.serialization;
+
+import java.io.Closeable;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ *
+ * @param <T> Type to be deserialized into.
+ *
+ * A class that implements this interface is expected to have a constructor with no parameter.
+ */
+public interface Deserializer<T> extends Closeable {
+
+    /**
+     * Configure this class.
+     * @param configs configs in key/value pairs
+     * @param isKey whether is for key or value
+     */
+    public void configure(Map<String, ?> configs, boolean isKey);
+    
+    /**
+     *
+     * @param topic topic associated with the data
+     * @param data serialized bytes
+     * @return deserialized typed data
+     */
+    public T deserialize(String topic, byte[] data);
+
+    @Override
+    public void close();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerDeserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerDeserializer.java
new file mode 100644
index 0000000..c5833d5
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerDeserializer.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.kafka_backport.common.serialization;
+
+import org.apache.flink.kafka_backport.common.errors.SerializationException;
+
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class IntegerDeserializer implements Deserializer<Integer> {
+
+    public void configure(Map<String, ?> configs, boolean isKey) {
+        // nothing to do
+    }
+
+    public Integer deserialize(String topic, byte[] data) {
+        if (data == null)
+            return null;
+        if (data.length != 4) {
+            throw new SerializationException("Size of data received by IntegerDeserializer is " +
+                    "not 4");
+        }
+
+        int value = 0;
+        for (byte b : data) {
+            value <<= 8;
+            value |= b & 0xFF;
+        }
+        return value;
+    }
+
+    public void close() {
+        // nothing to do
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerSerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerSerializer.java
new file mode 100644
index 0000000..dcbd7be
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerSerializer.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.kafka_backport.common.serialization;
+
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class IntegerSerializer implements Serializer<Integer> {
+
+    public void configure(Map<String, ?> configs, boolean isKey) {
+        // nothing to do
+    }
+
+    public byte[] serialize(String topic, Integer data) {
+        if (data == null)
+            return null;
+
+        return new byte[] {
+            (byte) (data >>> 24),
+            (byte) (data >>> 16),
+            (byte) (data >>> 8),
+            data.byteValue()
+        };
+    }
+
+    public void close() {
+        // nothing to do
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Serializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Serializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Serializer.java
new file mode 100644
index 0000000..1725e36
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Serializer.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.kafka_backport.common.serialization;
+
+import java.io.Closeable;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ *
+ * @param <T> Type to be serialized from.
+ *
+ * A class that implements this interface is expected to have a constructor with no parameter.
+ */
+public interface Serializer<T> extends Closeable {
+
+    /**
+     * Configure this class.
+     * @param configs configs in key/value pairs
+     * @param isKey whether is for key or value
+     */
+    public void configure(Map<String, ?> configs, boolean isKey);
+
+    /**
+     * @param topic topic associated with data
+     * @param data typed data
+     * @return serialized bytes
+     */
+    public byte[] serialize(String topic, T data);
+
+
+    /**
+     * Close this serializer.
+     * This method has to be idempotent if the serializer is used in KafkaProducer because it might be called
+     * multiple times.
+     */
+    @Override
+    public void close();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringDeserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringDeserializer.java
new file mode 100644
index 0000000..4d2ed4c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringDeserializer.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.kafka_backport.common.serialization;
+
+import org.apache.flink.kafka_backport.common.errors.SerializationException;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ *  String encoding defaults to UTF8 and can be customized by setting the property key.deserializer.encoding,
+ *  value.deserializer.encoding or deserializer.encoding. The first two take precedence over the last.
+ */
+public class StringDeserializer implements Deserializer<String> {
+    private String encoding = "UTF8";
+
+    @Override
+    public void configure(Map<String, ?> configs, boolean isKey) {
+        String propertyName = isKey ? "key.deserializer.encoding" : "value.deserializer.encoding";
+        Object encodingValue = configs.get(propertyName);
+        if (encodingValue == null)
+            encodingValue = configs.get("deserializer.encoding");
+        if (encodingValue != null && encodingValue instanceof String)
+            encoding = (String) encodingValue;
+    }
+
+    @Override
+    public String deserialize(String topic, byte[] data) {
+        try {
+            if (data == null)
+                return null;
+            else
+                return new String(data, encoding);
+        } catch (UnsupportedEncodingException e) {
+            throw new SerializationException("Error when deserializing byte[] to string due to unsupported encoding " + encoding);
+        }
+    }
+
+    @Override
+    public void close() {
+        // nothing to do
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringSerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringSerializer.java
new file mode 100644
index 0000000..fae4c21
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringSerializer.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.kafka_backport.common.serialization;
+
+import org.apache.flink.kafka_backport.common.errors.SerializationException;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ *  String encoding defaults to UTF8 and can be customized by setting the property key.serializer.encoding,
+ *  value.serializer.encoding or serializer.encoding. The first two take precedence over the last.
+ */
+public class StringSerializer implements Serializer<String> {
+    private String encoding = "UTF8";
+
+    @Override
+    public void configure(Map<String, ?> configs, boolean isKey) {
+        String propertyName = isKey ? "key.serializer.encoding" : "value.serializer.encoding";
+        Object encodingValue = configs.get(propertyName);
+        if (encodingValue == null)
+            encodingValue = configs.get("serializer.encoding");
+        if (encodingValue != null && encodingValue instanceof String)
+            encoding = (String) encodingValue;
+    }
+
+    @Override
+    public byte[] serialize(String topic, String data) {
+        try {
+            if (data == null)
+                return null;
+            else
+                return data.getBytes(encoding);
+        } catch (UnsupportedEncodingException e) {
+            throw new SerializationException("Error when serializing string to byte[] due to unsupported encoding " + encoding);
+        }
+    }
+
+    @Override
+    public void close() {
+        // nothing to do
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/AbstractIterator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/AbstractIterator.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/AbstractIterator.java
new file mode 100644
index 0000000..2af94b6
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/AbstractIterator.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.kafka_backport.common.utils;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A base class that simplifies implementing an iterator
+ * @param <T> The type of thing we are iterating over
+ */
+public abstract class AbstractIterator<T> implements Iterator<T> {
+
+    private static enum State {
+        READY, NOT_READY, DONE, FAILED
+    };
+
+    private State state = State.NOT_READY;
+    private T next;
+
+    @Override
+    public boolean hasNext() {
+        switch (state) {
+            case FAILED:
+                throw new IllegalStateException("Iterator is in failed state");
+            case DONE:
+                return false;
+            case READY:
+                return true;
+            default:
+                return maybeComputeNext();
+        }
+    }
+
+    @Override
+    public T next() {
+        if (!hasNext())
+            throw new NoSuchElementException();
+        state = State.NOT_READY;
+        if (next == null)
+            throw new IllegalStateException("Expected item but none found.");
+        return next;
+    }
+
+    @Override
+    public void remove() {
+        throw new UnsupportedOperationException("Removal not supported");
+    }
+
+    public T peek() {
+        if (!hasNext())
+            throw new NoSuchElementException();
+        return next;
+    }
+
+    protected T allDone() {
+        state = State.DONE;
+        return null;
+    }
+
+    protected abstract T makeNext();
+
+    private Boolean maybeComputeNext() {
+        state = State.FAILED;
+        next = makeNext();
+        if (state == State.DONE) {
+            return false;
+        } else {
+            state = State.READY;
+            return true;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CollectionUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CollectionUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CollectionUtils.java
new file mode 100644
index 0000000..7960331
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CollectionUtils.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.kafka_backport.common.utils;
+
+import org.apache.flink.kafka_backport.common.TopicPartition;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class CollectionUtils {
+    /**
+     * group data by topic
+     * @param data Data to be partitioned
+     * @param <T> Partition data type
+     * @return partitioned data
+     */
+    public static <T> Map<String, Map<Integer, T>> groupDataByTopic(Map<TopicPartition, T> data) {
+        Map<String, Map<Integer, T>> dataByTopic = new HashMap<String, Map<Integer, T>>();
+        for (Map.Entry<TopicPartition, T> entry: data.entrySet()) {
+            String topic = entry.getKey().topic();
+            int partition = entry.getKey().partition();
+            Map<Integer, T> topicData = dataByTopic.get(topic);
+            if (topicData == null) {
+                topicData = new HashMap<Integer, T>();
+                dataByTopic.put(topic, topicData);
+            }
+            topicData.put(partition, entry.getValue());
+        }
+        return dataByTopic;
+    }
+
+    /**
+     * group partitions by topic
+     * @param partitions
+     * @return partitions per topic
+     */
+    public static Map<String, List<Integer>> groupDataByTopic(List<TopicPartition> partitions) {
+        Map<String, List<Integer>> partitionsByTopic = new HashMap<String, List<Integer>>();
+        for (TopicPartition tp: partitions) {
+            String topic = tp.topic();
+            List<Integer> topicData = partitionsByTopic.get(topic);
+            if (topicData == null) {
+                topicData = new ArrayList<Integer>();
+                partitionsByTopic.put(topic, topicData);
+            }
+            topicData.add(tp.partition());
+        }
+        return  partitionsByTopic;
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CopyOnWriteMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CopyOnWriteMap.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CopyOnWriteMap.java
new file mode 100644
index 0000000..dcf219a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CopyOnWriteMap.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.kafka_backport.common.utils;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A simple read-optimized map implementation that synchronizes only writes and does a full copy on each modification
+ */
+public class CopyOnWriteMap<K, V> implements ConcurrentMap<K, V> {
+
+    private volatile Map<K, V> map;
+
+    public CopyOnWriteMap() {
+        this.map = Collections.emptyMap();
+    }
+
+    public CopyOnWriteMap(Map<K, V> map) {
+        this.map = Collections.unmodifiableMap(map);
+    }
+
+    @Override
+    public boolean containsKey(Object k) {
+        return map.containsKey(k);
+    }
+
+    @Override
+    public boolean containsValue(Object v) {
+        return map.containsValue(v);
+    }
+
+    @Override
+    public Set<Entry<K, V>> entrySet() {
+        return map.entrySet();
+    }
+
+    @Override
+    public V get(Object k) {
+        return map.get(k);
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return map.isEmpty();
+    }
+
+    @Override
+    public Set<K> keySet() {
+        return map.keySet();
+    }
+
+    @Override
+    public int size() {
+        return map.size();
+    }
+
+    @Override
+    public Collection<V> values() {
+        return map.values();
+    }
+
+    @Override
+    public synchronized void clear() {
+        this.map = Collections.emptyMap();
+    }
+
+    @Override
+    public synchronized V put(K k, V v) {
+        Map<K, V> copy = new HashMap<K, V>(this.map);
+        V prev = copy.put(k, v);
+        this.map = Collections.unmodifiableMap(copy);
+        return prev;
+    }
+
+    @Override
+    public synchronized void putAll(Map<? extends K, ? extends V> entries) {
+        Map<K, V> copy = new HashMap<K, V>(this.map);
+        copy.putAll(entries);
+        this.map = Collections.unmodifiableMap(copy);
+    }
+
+    @Override
+    public synchronized V remove(Object key) {
+        Map<K, V> copy = new HashMap<K, V>(this.map);
+        V prev = copy.remove(key);
+        this.map = Collections.unmodifiableMap(copy);
+        return prev;
+    }
+
+    @Override
+    public synchronized V putIfAbsent(K k, V v) {
+        if (!containsKey(k))
+            return put(k, v);
+        else
+            return get(k);
+    }
+
+    @Override
+    public synchronized boolean remove(Object k, Object v) {
+        if (containsKey(k) && get(k).equals(v)) {
+            remove(k);
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public synchronized boolean replace(K k, V original, V replacement) {
+        if (containsKey(k) && get(k).equals(original)) {
+            put(k, replacement);
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public synchronized V replace(K k, V v) {
+        if (containsKey(k)) {
+            return put(k, v);
+        } else {
+            return null;
+        }
+    }
+
+}


[33/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.java
new file mode 100644
index 0000000..c412136
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.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.internals;
+
+import kafka.admin.AdminUtils;
+
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.flink.streaming.connectors.KafkaTestBase;
+
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class ZookeeperOffsetHandlerTest extends KafkaTestBase {
+	
+	@Test
+	public void runOffsetManipulationinZooKeeperTest() {
+		try {
+			final String topicName = "ZookeeperOffsetHandlerTest-Topic";
+			final String groupId = "ZookeeperOffsetHandlerTest-Group";
+			
+			final long offset = (long) (Math.random() * Long.MAX_VALUE);
+
+			ZkClient zkClient = createZookeeperClient();
+			AdminUtils.createTopic(zkClient, topicName, 3, 2, new Properties());
+				
+			ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, groupId, topicName, 0, offset);
+	
+			long fetchedOffset = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, groupId, topicName, 0);
+
+			zkClient.close();
+			
+			assertEquals(offset, fetchedOffset);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}


[09/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/ConsumerNetworkClient.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/ConsumerNetworkClient.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/ConsumerNetworkClient.java
deleted file mode 100644
index 5499a5d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/ConsumerNetworkClient.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer.internals;
-
-import org.apache.kafka.copied.clients.ClientRequest;
-import org.apache.kafka.copied.clients.ClientResponse;
-import org.apache.kafka.copied.clients.KafkaClient;
-import org.apache.kafka.copied.clients.Metadata;
-import org.apache.kafka.copied.clients.RequestCompletionHandler;
-import org.apache.kafka.copied.clients.consumer.ConsumerWakeupException;
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.requests.AbstractRequest;
-import org.apache.kafka.copied.common.requests.RequestHeader;
-import org.apache.kafka.copied.common.requests.RequestSend;
-import org.apache.kafka.copied.common.utils.Time;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-/**
- * Higher level consumer access to the network layer with basic support for futures and
- * task scheduling. NOT thread-safe!
- *
- * TODO: The current implementation is simplistic in that it provides a facility for queueing requests
- * prior to delivery, but it makes no effort to retry requests which cannot be sent at the time
- * {@link #poll(long)} is called. This makes the behavior of the queue predictable and easy to
- * understand, but there are opportunities to provide timeout or retry capabilities in the future.
- * How we do this may depend on KAFKA-2120, so for now, we retain the simplistic behavior.
- */
-public class ConsumerNetworkClient implements Closeable {
-    private final KafkaClient client;
-    private final AtomicBoolean wakeup = new AtomicBoolean(false);
-    private final DelayedTaskQueue delayedTasks = new DelayedTaskQueue();
-    private final Map<Node, List<ClientRequest>> unsent = new HashMap<Node, List<ClientRequest>>();
-    private final Metadata metadata;
-    private final Time time;
-    private final long retryBackoffMs;
-
-    public ConsumerNetworkClient(KafkaClient client,
-                                 Metadata metadata,
-                                 Time time,
-                                 long retryBackoffMs) {
-        this.client = client;
-        this.metadata = metadata;
-        this.time = time;
-        this.retryBackoffMs = retryBackoffMs;
-    }
-
-    /**
-     * Schedule a new task to be executed at the given time. This is "best-effort" scheduling and
-     * should only be used for coarse synchronization.
-     * @param task The task to be scheduled
-     * @param at The time it should run
-     */
-    public void schedule(DelayedTask task, long at) {
-        delayedTasks.add(task, at);
-    }
-
-    /**
-     * Unschedule a task. This will remove all instances of the task from the task queue.
-     * This is a no-op if the task is not scheduled.
-     * @param task The task to be unscheduled.
-     */
-    public void unschedule(DelayedTask task) {
-        delayedTasks.remove(task);
-    }
-
-    /**
-     * Send a new request. Note that the request is not actually transmitted on the
-     * network until one of the {@link #poll(long)} variants is invoked. At this
-     * point the request will either be transmitted successfully or will fail.
-     * Use the returned future to obtain the result of the send.
-     * @param node The destination of the request
-     * @param api The Kafka API call
-     * @param request The request payload
-     * @return A future which indicates the result of the send.
-     */
-    public RequestFuture<ClientResponse> send(Node node,
-                                              ApiKeys api,
-                                              AbstractRequest request) {
-        long now = time.milliseconds();
-        RequestFutureCompletionHandler future = new RequestFutureCompletionHandler();
-        RequestHeader header = client.nextRequestHeader(api);
-        RequestSend send = new RequestSend(node.idString(), header, request.toStruct());
-        put(node, new ClientRequest(now, true, send, future));
-        return future;
-    }
-
-    private void put(Node node, ClientRequest request) {
-        List<ClientRequest> nodeUnsent = unsent.get(node);
-        if (nodeUnsent == null) {
-            nodeUnsent = new ArrayList<ClientRequest>();
-            unsent.put(node, nodeUnsent);
-        }
-        nodeUnsent.add(request);
-    }
-
-    public Node leastLoadedNode() {
-        return client.leastLoadedNode(time.milliseconds());
-    }
-
-    /**
-     * Block until the metadata has been refreshed.
-     */
-    public void awaitMetadataUpdate() {
-        int version = this.metadata.requestUpdate();
-        do {
-            poll(Long.MAX_VALUE);
-        } while (this.metadata.version() == version);
-    }
-
-    /**
-     * Wakeup an active poll. This will cause the polling thread to throw an exception either
-     * on the current poll if one is active, or the next poll.
-     */
-    public void wakeup() {
-        this.wakeup.set(true);
-        this.client.wakeup();
-    }
-
-    /**
-     * Block indefinitely until the given request future has finished.
-     * @param future The request future to await.
-     * @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread
-     */
-    public void poll(RequestFuture<?> future) {
-        while (!future.isDone())
-            poll(Long.MAX_VALUE);
-    }
-
-    /**
-     * Block until the provided request future request has finished or the timeout has expired.
-     * @param future The request future to wait for
-     * @param timeout The maximum duration (in ms) to wait for the request
-     * @return true if the future is done, false otherwise
-     * @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread
-     */
-    public boolean poll(RequestFuture<?> future, long timeout) {
-        long now = time.milliseconds();
-        long deadline = now + timeout;
-        while (!future.isDone() && now < deadline) {
-            poll(deadline - now, now);
-            now = time.milliseconds();
-        }
-        return future.isDone();
-    }
-
-    /**
-     * Poll for any network IO. All send requests will either be transmitted on the network
-     * or failed when this call completes.
-     * @param timeout The maximum time to wait for an IO event.
-     * @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread
-     */
-    public void poll(long timeout) {
-        poll(timeout, time.milliseconds());
-    }
-
-    private void poll(long timeout, long now) {
-        // send all the requests we can send now
-        pollUnsentRequests(now);
-
-        // ensure we don't poll any longer than the deadline for
-        // the next scheduled task
-        timeout = Math.min(timeout, delayedTasks.nextTimeout(now));
-        clientPoll(timeout, now);
-
-        // execute scheduled tasks
-        now = time.milliseconds();
-        delayedTasks.poll(now);
-
-        // try again to send requests since buffer space may have been
-        // cleared or a connect finished in the poll
-        pollUnsentRequests(now);
-
-        // fail all requests that couldn't be sent
-        clearUnsentRequests(now);
-
-    }
-
-    /**
-     * Block until all pending requests from the given node have finished.
-     * @param node The node to await requests from
-     */
-    public void awaitPendingRequests(Node node) {
-        while (pendingRequestCount(node) > 0)
-            poll(retryBackoffMs);
-    }
-
-    /**
-     * Get the count of pending requests to the given node. This includes both request that
-     * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission.
-     * @param node The node in question
-     * @return The number of pending requests
-     */
-    public int pendingRequestCount(Node node) {
-        List<ClientRequest> pending = unsent.get(node);
-        int unsentCount = pending == null ? 0 : pending.size();
-        return unsentCount + client.inFlightRequestCount(node.idString());
-    }
-
-    /**
-     * Get the total count of pending requests from all nodes. This includes both requests that
-     * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission.
-     * @return The total count of pending requests
-     */
-    public int pendingRequestCount() {
-        int total = 0;
-        for (List<ClientRequest> requests: unsent.values())
-            total += requests.size();
-        return total + client.inFlightRequestCount();
-    }
-
-    private void pollUnsentRequests(long now) {
-        while (trySend(now))
-            clientPoll(0, now);
-    }
-
-    private void clearUnsentRequests(long now) {
-        // clear all unsent requests and fail their corresponding futures
-        for (Map.Entry<Node, List<ClientRequest>> requestEntry: unsent.entrySet()) {
-            Iterator<ClientRequest> iterator = requestEntry.getValue().iterator();
-            while (iterator.hasNext()) {
-                ClientRequest request = iterator.next();
-                RequestFutureCompletionHandler handler =
-                        (RequestFutureCompletionHandler) request.callback();
-                handler.raise(SendFailedException.INSTANCE);
-                iterator.remove();
-            }
-        }
-        unsent.clear();
-    }
-
-    private boolean trySend(long now) {
-        // send any requests that can be sent now
-        boolean requestsSent = false;
-        for (Map.Entry<Node, List<ClientRequest>> requestEntry: unsent.entrySet()) {
-            Node node = requestEntry.getKey();
-            Iterator<ClientRequest> iterator = requestEntry.getValue().iterator();
-            while (iterator.hasNext()) {
-                ClientRequest request = iterator.next();
-                if (client.ready(node, now)) {
-                    client.send(request);
-                    iterator.remove();
-                    requestsSent = true;
-                } else if (client.connectionFailed(node)) {
-                    RequestFutureCompletionHandler handler =
-                            (RequestFutureCompletionHandler) request.callback();
-                    handler.onComplete(new ClientResponse(request, now, true, null));
-                    iterator.remove();
-                }
-            }
-        }
-        return requestsSent;
-    }
-
-    private void clientPoll(long timeout, long now) {
-        client.poll(timeout, now);
-        if (wakeup.get()) {
-            clearUnsentRequests(now);
-            wakeup.set(false);
-            throw new ConsumerWakeupException();
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        client.close();
-    }
-
-    public static class RequestFutureCompletionHandler
-            extends RequestFuture<ClientResponse>
-            implements RequestCompletionHandler {
-
-        @Override
-        public void onComplete(ClientResponse response) {
-            complete(response);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Coordinator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Coordinator.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Coordinator.java
deleted file mode 100644
index b28f6bc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Coordinator.java
+++ /dev/null
@@ -1,791 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer.internals;
-
-import org.apache.kafka.copied.clients.ClientResponse;
-import org.apache.kafka.copied.clients.consumer.CommitType;
-import org.apache.kafka.copied.clients.consumer.ConsumerCommitCallback;
-import org.apache.kafka.copied.common.KafkaException;
-import org.apache.kafka.copied.common.MetricName;
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.errors.DisconnectException;
-import org.apache.kafka.copied.common.metrics.Measurable;
-import org.apache.kafka.copied.common.metrics.MetricConfig;
-import org.apache.kafka.copied.common.metrics.Metrics;
-import org.apache.kafka.copied.common.metrics.Sensor;
-import org.apache.kafka.copied.common.metrics.stats.Avg;
-import org.apache.kafka.copied.common.metrics.stats.Count;
-import org.apache.kafka.copied.common.metrics.stats.Max;
-import org.apache.kafka.copied.common.metrics.stats.Rate;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.requests.ConsumerMetadataRequest;
-import org.apache.kafka.copied.common.requests.ConsumerMetadataResponse;
-import org.apache.kafka.copied.common.requests.HeartbeatRequest;
-import org.apache.kafka.copied.common.requests.HeartbeatResponse;
-import org.apache.kafka.copied.common.requests.JoinGroupRequest;
-import org.apache.kafka.copied.common.requests.JoinGroupResponse;
-import org.apache.kafka.copied.common.requests.OffsetCommitRequest;
-import org.apache.kafka.copied.common.requests.OffsetCommitResponse;
-import org.apache.kafka.copied.common.requests.OffsetFetchRequest;
-import org.apache.kafka.copied.common.requests.OffsetFetchResponse;
-import org.apache.kafka.copied.common.utils.Time;
-import org.apache.kafka.copied.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-/**
- * This class manages the coordination process with the consumer coordinator.
- */
-public final class Coordinator {
-
-    private static final Logger log = LoggerFactory.getLogger(Coordinator.class);
-
-    private final ConsumerNetworkClient client;
-    private final Time time;
-    private final String groupId;
-    private final Heartbeat heartbeat;
-    private final HeartbeatTask heartbeatTask;
-    private final int sessionTimeoutMs;
-    private final String assignmentStrategy;
-    private final SubscriptionState subscriptions;
-    private final CoordinatorMetrics sensors;
-    private final long requestTimeoutMs;
-    private final long retryBackoffMs;
-    private final RebalanceCallback rebalanceCallback;
-    private Node consumerCoordinator;
-    private String consumerId;
-    private int generation;
-
-
-    /**
-     * Initialize the coordination manager.
-     */
-    public Coordinator(ConsumerNetworkClient client,
-                       String groupId,
-                       int sessionTimeoutMs,
-                       String assignmentStrategy,
-                       SubscriptionState subscriptions,
-                       Metrics metrics,
-                       String metricGrpPrefix,
-                       Map<String, String> metricTags,
-                       Time time,
-                       long requestTimeoutMs,
-                       long retryBackoffMs,
-                       RebalanceCallback rebalanceCallback) {
-
-        this.client = client;
-        this.time = time;
-        this.generation = -1;
-        this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-        this.groupId = groupId;
-        this.consumerCoordinator = null;
-        this.subscriptions = subscriptions;
-        this.sessionTimeoutMs = sessionTimeoutMs;
-        this.assignmentStrategy = assignmentStrategy;
-        this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds());
-        this.heartbeatTask = new HeartbeatTask();
-        this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
-        this.requestTimeoutMs = requestTimeoutMs;
-        this.retryBackoffMs = retryBackoffMs;
-        this.rebalanceCallback = rebalanceCallback;
-    }
-
-    /**
-     * Refresh the committed offsets for provided partitions.
-     */
-    public void refreshCommittedOffsetsIfNeeded() {
-        if (subscriptions.refreshCommitsNeeded()) {
-            Map<TopicPartition, Long> offsets = fetchCommittedOffsets(subscriptions.assignedPartitions());
-            for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet()) {
-                TopicPartition tp = entry.getKey();
-                this.subscriptions.committed(tp, entry.getValue());
-            }
-            this.subscriptions.commitsRefreshed();
-        }
-    }
-
-    /**
-     * Fetch the current committed offsets from the coordinator for a set of partitions.
-     * @param partitions The partitions to fetch offsets for
-     * @return A map from partition to the committed offset
-     */
-    public Map<TopicPartition, Long> fetchCommittedOffsets(Set<TopicPartition> partitions) {
-        while (true) {
-            ensureCoordinatorKnown();
-            ensurePartitionAssignment();
-
-            // contact coordinator to fetch committed offsets
-            RequestFuture<Map<TopicPartition, Long>> future = sendOffsetFetchRequest(partitions);
-            client.poll(future);
-
-            if (future.succeeded())
-                return future.value();
-
-            if (!future.isRetriable())
-                throw future.exception();
-
-            Utils.sleep(retryBackoffMs);
-        }
-    }
-
-    /**
-     * Ensure that we have a valid partition assignment from the coordinator.
-     */
-    public void ensurePartitionAssignment() {
-        if (!subscriptions.partitionAssignmentNeeded())
-            return;
-
-        // execute the user's callback before rebalance
-        log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions());
-        try {
-            Set<TopicPartition> revoked = new HashSet<TopicPartition>(subscriptions.assignedPartitions());
-            rebalanceCallback.onPartitionsRevoked(revoked);
-        } catch (Exception e) {
-            log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
-                    + " failed on partition revocation: ", e);
-        }
-
-        reassignPartitions();
-
-        // execute the user's callback after rebalance
-        log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions());
-        try {
-            Set<TopicPartition> assigned = new HashSet<TopicPartition>(subscriptions.assignedPartitions());
-            rebalanceCallback.onPartitionsAssigned(assigned);
-        } catch (Exception e) {
-            log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
-                    + " failed on partition assignment: ", e);
-        }
-    }
-
-    private void reassignPartitions() {
-        while (subscriptions.partitionAssignmentNeeded()) {
-            ensureCoordinatorKnown();
-
-            // ensure that there are no pending requests to the coordinator. This is important
-            // in particular to avoid resending a pending JoinGroup request.
-            if (client.pendingRequestCount(this.consumerCoordinator) > 0) {
-                client.awaitPendingRequests(this.consumerCoordinator);
-                continue;
-            }
-
-            RequestFuture<Void> future = sendJoinGroupRequest();
-            client.poll(future);
-
-            if (future.failed()) {
-                if (!future.isRetriable())
-                    throw future.exception();
-                Utils.sleep(retryBackoffMs);
-            }
-        }
-    }
-
-    /**
-     * Block until the coordinator for this group is known.
-     */
-    public void ensureCoordinatorKnown() {
-        while (coordinatorUnknown()) {
-            RequestFuture<Void> future = sendConsumerMetadataRequest();
-            client.poll(future, requestTimeoutMs);
-
-            if (future.failed())
-                client.awaitMetadataUpdate();
-        }
-    }
-
-    /**
-     * Commit offsets. This call blocks (regardless of commitType) until the coordinator
-     * can receive the commit request. Once the request has been made, however, only the
-     * synchronous commits will wait for a successful response from the coordinator.
-     * @param offsets Offsets to commit.
-     * @param commitType Commit policy
-     * @param callback Callback to be executed when the commit request finishes
-     */
-    public void commitOffsets(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
-        if (commitType == CommitType.ASYNC)
-            commitOffsetsAsync(offsets, callback);
-        else
-            commitOffsetsSync(offsets, callback);
-    }
-
-    private class HeartbeatTask implements DelayedTask {
-
-        public void reset() {
-            // start or restart the heartbeat task to be executed at the next chance
-            long now = time.milliseconds();
-            heartbeat.resetSessionTimeout(now);
-            client.unschedule(this);
-            client.schedule(this, now);
-        }
-
-        @Override
-        public void run(final long now) {
-            if (!subscriptions.partitionsAutoAssigned() ||
-                    subscriptions.partitionAssignmentNeeded() ||
-                    coordinatorUnknown())
-                // no need to send if we're not using auto-assignment or if we are
-                // awaiting a rebalance
-                return;
-
-            if (heartbeat.sessionTimeoutExpired(now)) {
-                // we haven't received a successful heartbeat in one session interval
-                // so mark the coordinator dead
-                coordinatorDead();
-                return;
-            }
-
-            if (!heartbeat.shouldHeartbeat(now)) {
-                // we don't need to heartbeat now, so reschedule for when we do
-                client.schedule(this, now + heartbeat.timeToNextHeartbeat(now));
-            } else {
-                heartbeat.sentHeartbeat(now);
-                RequestFuture<Void> future = sendHeartbeatRequest();
-                future.addListener(new RequestFutureListener<Void>() {
-                    @Override
-                    public void onSuccess(Void value) {
-                        long now = time.milliseconds();
-                        heartbeat.receiveHeartbeat(now);
-                        long nextHeartbeatTime = now + heartbeat.timeToNextHeartbeat(now);
-                        client.schedule(HeartbeatTask.this, nextHeartbeatTime);
-                    }
-
-                    @Override
-                    public void onFailure(RuntimeException e) {
-                        client.schedule(HeartbeatTask.this, time.milliseconds() + retryBackoffMs);
-                    }
-                });
-            }
-        }
-    }
-
-    /**
-     * Send a request to get a new partition assignment. This is a non-blocking call which sends
-     * a JoinGroup request to the coordinator (if it is available). The returned future must
-     * be polled to see if the request completed successfully.
-     * @return A request future whose completion indicates the result of the JoinGroup request.
-     */
-    private RequestFuture<Void> sendJoinGroupRequest() {
-        if (coordinatorUnknown())
-            return RequestFuture.coordinatorNotAvailable();
-
-        // send a join group request to the coordinator
-        List<String> subscribedTopics = new ArrayList<String>(subscriptions.subscribedTopics());
-        log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics);
-
-        JoinGroupRequest request = new JoinGroupRequest(groupId,
-                this.sessionTimeoutMs,
-                subscribedTopics,
-                this.consumerId,
-                this.assignmentStrategy);
-
-        // create the request for the coordinator
-        log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id());
-        return client.send(consumerCoordinator, ApiKeys.JOIN_GROUP, request)
-                .compose(new JoinGroupResponseHandler());
-    }
-
-    private class JoinGroupResponseHandler extends CoordinatorResponseHandler<JoinGroupResponse, Void> {
-
-        @Override
-        public JoinGroupResponse parse(ClientResponse response) {
-            return new JoinGroupResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(JoinGroupResponse joinResponse, RequestFuture<Void> future) {
-            // process the response
-            short errorCode = joinResponse.errorCode();
-
-            if (errorCode == Errors.NONE.code()) {
-                Coordinator.this.consumerId = joinResponse.consumerId();
-                Coordinator.this.generation = joinResponse.generationId();
-
-                // set the flag to refresh last committed offsets
-                subscriptions.needRefreshCommits();
-
-                log.debug("Joined group: {}", joinResponse.toStruct());
-
-                // record re-assignment time
-                sensors.partitionReassignments.record(response.requestLatencyMs());
-
-                // update partition assignment
-                subscriptions.changePartitionAssignment(joinResponse.assignedPartitions());
-                heartbeatTask.reset();
-                future.complete(null);
-            } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) {
-                // reset the consumer id and retry immediately
-                Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-                log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.",
-                        groupId);
-                future.raise(Errors.UNKNOWN_CONSUMER_ID);
-            } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
-                    || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                // re-discover the coordinator and retry with backoff
-                coordinatorDead();
-                log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.",
-                        groupId);
-                future.raise(Errors.forCode(errorCode));
-            } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code()
-                    || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code()
-                    || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) {
-                // log the error and re-throw the exception
-                Errors error = Errors.forCode(errorCode);
-                log.error("Attempt to join group {} failed due to: {}",
-                        groupId, error.exception().getMessage());
-                future.raise(error);
-            } else {
-                // unexpected error, throw the exception
-                future.raise(new KafkaException("Unexpected error in join group response: "
-                        + Errors.forCode(joinResponse.errorCode()).exception().getMessage()));
-            }
-        }
-    }
-
-    private void commitOffsetsAsync(final Map<TopicPartition, Long> offsets, final ConsumerCommitCallback callback) {
-        this.subscriptions.needRefreshCommits();
-        RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
-        if (callback != null) {
-            future.addListener(new RequestFutureListener<Void>() {
-                @Override
-                public void onSuccess(Void value) {
-                    callback.onComplete(offsets, null);
-                }
-
-                @Override
-                public void onFailure(RuntimeException e) {
-                    callback.onComplete(offsets, e);
-                }
-            });
-        }
-    }
-
-    private void commitOffsetsSync(Map<TopicPartition, Long> offsets, ConsumerCommitCallback callback) {
-        while (true) {
-            ensureCoordinatorKnown();
-            ensurePartitionAssignment();
-
-            RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
-            client.poll(future);
-
-            if (future.succeeded()) {
-                if (callback != null)
-                    callback.onComplete(offsets, null);
-                return;
-            }
-
-            if (!future.isRetriable()) {
-                if (callback == null)
-                    throw future.exception();
-                else
-                    callback.onComplete(offsets, future.exception());
-                return;
-            }
-
-            Utils.sleep(retryBackoffMs);
-        }
-    }
-
-    /**
-     * Commit offsets for the specified list of topics and partitions. This is a non-blocking call
-     * which returns a request future that can be polled in the case of a synchronous commit or ignored in the
-     * asynchronous case.
-     *
-     * @param offsets The list of offsets per partition that should be committed.
-     * @return A request future whose value indicates whether the commit was successful or not
-     */
-    private RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, Long> offsets) {
-        if (coordinatorUnknown())
-            return RequestFuture.coordinatorNotAvailable();
-
-        if (offsets.isEmpty())
-            return RequestFuture.voidSuccess();
-
-        // create the offset commit request
-        Map<TopicPartition, OffsetCommitRequest.PartitionData> offsetData;
-        offsetData = new HashMap<TopicPartition, OffsetCommitRequest.PartitionData>(offsets.size());
-        for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet())
-            offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), ""));
-        OffsetCommitRequest req = new OffsetCommitRequest(this.groupId,
-                this.generation,
-                this.consumerId,
-                OffsetCommitRequest.DEFAULT_RETENTION_TIME,
-                offsetData);
-
-        return client.send(consumerCoordinator, ApiKeys.OFFSET_COMMIT, req)
-                .compose(new OffsetCommitResponseHandler(offsets));
-    }
-
-
-    private class OffsetCommitResponseHandler extends CoordinatorResponseHandler<OffsetCommitResponse, Void> {
-
-        private final Map<TopicPartition, Long> offsets;
-
-        public OffsetCommitResponseHandler(Map<TopicPartition, Long> offsets) {
-            this.offsets = offsets;
-        }
-
-        @Override
-        public OffsetCommitResponse parse(ClientResponse response) {
-            return new OffsetCommitResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> future) {
-            sensors.commitLatency.record(response.requestLatencyMs());
-            for (Map.Entry<TopicPartition, Short> entry : commitResponse.responseData().entrySet()) {
-                TopicPartition tp = entry.getKey();
-                long offset = this.offsets.get(tp);
-                short errorCode = entry.getValue();
-                if (errorCode == Errors.NONE.code()) {
-                    log.debug("Committed offset {} for partition {}", offset, tp);
-                    subscriptions.committed(tp, offset);
-                } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
-                        || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                    coordinatorDead();
-                    future.raise(Errors.forCode(errorCode));
-                    return;
-                } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code()
-                        || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) {
-                    // do not need to throw the exception but just log the error
-                    log.error("Error committing partition {} at offset {}: {}",
-                            tp,
-                            offset,
-                            Errors.forCode(errorCode).exception().getMessage());
-                } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
-                        || errorCode == Errors.ILLEGAL_GENERATION.code()) {
-                    // need to re-join group
-                    subscriptions.needReassignment();
-                    future.raise(Errors.forCode(errorCode));
-                    return;
-                } else {
-                    // do not need to throw the exception but just log the error
-                    future.raise(Errors.forCode(errorCode));
-                    log.error("Error committing partition {} at offset {}: {}",
-                            tp,
-                            offset,
-                            Errors.forCode(errorCode).exception().getMessage());
-                }
-            }
-
-            future.complete(null);
-        }
-    }
-
-    /**
-     * Fetch the committed offsets for a set of partitions. This is a non-blocking call. The
-     * returned future can be polled to get the actual offsets returned from the broker.
-     *
-     * @param partitions The set of partitions to get offsets for.
-     * @return A request future containing the committed offsets.
-     */
-    private RequestFuture<Map<TopicPartition, Long>> sendOffsetFetchRequest(Set<TopicPartition> partitions) {
-        if (coordinatorUnknown())
-            return RequestFuture.coordinatorNotAvailable();
-
-        log.debug("Fetching committed offsets for partitions: {}",  Utils.join(partitions, ", "));
-        // construct the request
-        OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList<TopicPartition>(partitions));
-
-        // send the request with a callback
-        return client.send(consumerCoordinator, ApiKeys.OFFSET_FETCH, request)
-                .compose(new OffsetFetchResponseHandler());
-    }
-
-    private class OffsetFetchResponseHandler extends CoordinatorResponseHandler<OffsetFetchResponse, Map<TopicPartition, Long>> {
-
-        @Override
-        public OffsetFetchResponse parse(ClientResponse response) {
-            return new OffsetFetchResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, Long>> future) {
-            Map<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>(response.responseData().size());
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) {
-                TopicPartition tp = entry.getKey();
-                OffsetFetchResponse.PartitionData data = entry.getValue();
-                if (data.hasError()) {
-                    log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode)
-                            .exception()
-                            .getMessage());
-                    if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) {
-                        // just retry
-                        future.raise(Errors.OFFSET_LOAD_IN_PROGRESS);
-                    } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                        // re-discover the coordinator and retry
-                        coordinatorDead();
-                        future.raise(Errors.NOT_COORDINATOR_FOR_CONSUMER);
-                    } else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
-                            || data.errorCode == Errors.ILLEGAL_GENERATION.code()) {
-                        // need to re-join group
-                        subscriptions.needReassignment();
-                        future.raise(Errors.forCode(data.errorCode));
-                    } else {
-                        future.raise(new KafkaException("Unexpected error in fetch offset response: "
-                                + Errors.forCode(data.errorCode).exception().getMessage()));
-                    }
-                    return;
-                } else if (data.offset >= 0) {
-                    // record the position with the offset (-1 indicates no committed offset to fetch)
-                    offsets.put(tp, data.offset);
-                } else {
-                    log.debug("No committed offset for partition " + tp);
-                }
-            }
-
-            future.complete(offsets);
-        }
-    }
-
-    /**
-     * Send a heartbeat request now (visible only for testing).
-     */
-    public RequestFuture<Void> sendHeartbeatRequest() {
-        HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId);
-        return client.send(consumerCoordinator, ApiKeys.HEARTBEAT, req)
-                .compose(new HeartbeatCompletionHandler());
-    }
-
-    public boolean coordinatorUnknown() {
-        return this.consumerCoordinator == null;
-    }
-
-    /**
-     * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to
-     * one of the brokers. The returned future should be polled to get the result of the request.
-     * @return A request future which indicates the completion of the metadata request
-     */
-    private RequestFuture<Void> sendConsumerMetadataRequest() {
-        // initiate the consumer metadata request
-        // find a node to ask about the coordinator
-        Node node = this.client.leastLoadedNode();
-        if (node == null) {
-            // TODO: If there are no brokers left, perhaps we should use the bootstrap set
-            // from configuration?
-            return RequestFuture.noBrokersAvailable();
-        } else {
-            // create a consumer metadata request
-            log.debug("Issuing consumer metadata request to broker {}", node.id());
-            ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId);
-            return client.send(node, ApiKeys.CONSUMER_METADATA, metadataRequest)
-                    .compose(new RequestFutureAdapter<ClientResponse, Void>() {
-                        @Override
-                        public void onSuccess(ClientResponse response, RequestFuture<Void> future) {
-                            handleConsumerMetadataResponse(response, future);
-                        }
-                    });
-        }
-    }
-
-    private void handleConsumerMetadataResponse(ClientResponse resp, RequestFuture<Void> future) {
-        log.debug("Consumer metadata response {}", resp);
-
-        // parse the response to get the coordinator info if it is not disconnected,
-        // otherwise we need to request metadata update
-        if (resp.wasDisconnected()) {
-            future.raise(new DisconnectException());
-        } else if (!coordinatorUnknown()) {
-            // We already found the coordinator, so ignore the request
-            future.complete(null);
-        } else {
-            ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody());
-            // use MAX_VALUE - node.id as the coordinator id to mimic separate connections
-            // for the coordinator in the underlying network client layer
-            // TODO: this needs to be better handled in KAFKA-1935
-            if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) {
-                this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(),
-                        consumerMetadataResponse.node().host(),
-                        consumerMetadataResponse.node().port());
-                heartbeatTask.reset();
-                future.complete(null);
-            } else {
-                future.raise(Errors.forCode(consumerMetadataResponse.errorCode()));
-            }
-        }
-    }
-
-    /**
-     * Mark the current coordinator as dead.
-     */
-    private void coordinatorDead() {
-        if (this.consumerCoordinator != null) {
-            log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id());
-            this.consumerCoordinator = null;
-        }
-    }
-
-    private class HeartbeatCompletionHandler extends CoordinatorResponseHandler<HeartbeatResponse, Void> {
-        @Override
-        public HeartbeatResponse parse(ClientResponse response) {
-            return new HeartbeatResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(HeartbeatResponse heartbeatResponse, RequestFuture<Void> future) {
-            sensors.heartbeatLatency.record(response.requestLatencyMs());
-            short error = heartbeatResponse.errorCode();
-            if (error == Errors.NONE.code()) {
-                log.debug("Received successful heartbeat response.");
-                future.complete(null);
-            } else if (error == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
-                    || error == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead.");
-                coordinatorDead();
-                future.raise(Errors.forCode(error));
-            } else if (error == Errors.ILLEGAL_GENERATION.code()) {
-                log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group.");
-                subscriptions.needReassignment();
-                future.raise(Errors.ILLEGAL_GENERATION);
-            } else if (error == Errors.UNKNOWN_CONSUMER_ID.code()) {
-                log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group.");
-                consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-                subscriptions.needReassignment();
-                future.raise(Errors.UNKNOWN_CONSUMER_ID);
-            } else {
-                future.raise(new KafkaException("Unexpected error in heartbeat response: "
-                        + Errors.forCode(error).exception().getMessage()));
-            }
-        }
-    }
-
-    private abstract class CoordinatorResponseHandler<R, T>
-            extends RequestFutureAdapter<ClientResponse, T> {
-        protected ClientResponse response;
-
-        public abstract R parse(ClientResponse response);
-
-        public abstract void handle(R response, RequestFuture<T> future);
-
-        @Override
-        public void onSuccess(ClientResponse clientResponse, RequestFuture<T> future) {
-            this.response = clientResponse;
-
-            if (clientResponse.wasDisconnected()) {
-                int correlation = response.request().request().header().correlationId();
-                log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected",
-                        response.request(),
-                        correlation,
-                        response.request().request().destination());
-
-                // mark the coordinator as dead
-                coordinatorDead();
-                future.raise(new DisconnectException());
-                return;
-            }
-
-            R response = parse(clientResponse);
-            handle(response, future);
-        }
-
-        @Override
-        public void onFailure(RuntimeException e, RequestFuture<T> future) {
-            if (e instanceof DisconnectException) {
-                log.debug("Coordinator request failed", e);
-                coordinatorDead();
-            }
-            future.raise(e);
-        }
-    }
-
-    public interface RebalanceCallback {
-        void onPartitionsAssigned(Collection<TopicPartition> partitions);
-        void onPartitionsRevoked(Collection<TopicPartition> partitions);
-    }
-
-    private class CoordinatorMetrics {
-        public final Metrics metrics;
-        public final String metricGrpName;
-
-        public final Sensor commitLatency;
-        public final Sensor heartbeatLatency;
-        public final Sensor partitionReassignments;
-
-        public CoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
-            this.metrics = metrics;
-            this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
-
-            this.commitLatency = metrics.sensor("commit-latency");
-            this.commitLatency.add(new MetricName("commit-latency-avg",
-                this.metricGrpName,
-                "The average time taken for a commit request",
-                tags), new Avg());
-            this.commitLatency.add(new MetricName("commit-latency-max",
-                this.metricGrpName,
-                "The max time taken for a commit request",
-                tags), new Max());
-            this.commitLatency.add(new MetricName("commit-rate",
-                this.metricGrpName,
-                "The number of commit calls per second",
-                tags), new Rate(new Count()));
-
-            this.heartbeatLatency = metrics.sensor("heartbeat-latency");
-            this.heartbeatLatency.add(new MetricName("heartbeat-response-time-max",
-                this.metricGrpName,
-                "The max time taken to receive a response to a hearbeat request",
-                tags), new Max());
-            this.heartbeatLatency.add(new MetricName("heartbeat-rate",
-                this.metricGrpName,
-                "The average number of heartbeats per second",
-                tags), new Rate(new Count()));
-
-            this.partitionReassignments = metrics.sensor("reassignment-latency");
-            this.partitionReassignments.add(new MetricName("reassignment-time-avg",
-                this.metricGrpName,
-                "The average time taken for a partition reassignment",
-                tags), new Avg());
-            this.partitionReassignments.add(new MetricName("reassignment-time-max",
-                this.metricGrpName,
-                "The max time taken for a partition reassignment",
-                tags), new Avg());
-            this.partitionReassignments.add(new MetricName("reassignment-rate",
-                this.metricGrpName,
-                "The number of partition reassignments per second",
-                tags), new Rate(new Count()));
-
-            Measurable numParts =
-                new Measurable() {
-                    public double measure(MetricConfig config, long now) {
-                        return subscriptions.assignedPartitions().size();
-                    }
-                };
-            metrics.addMetric(new MetricName("assigned-partitions",
-                this.metricGrpName,
-                "The number of partitions currently assigned to this consumer",
-                tags),
-                numParts);
-
-            Measurable lastHeartbeat =
-                new Measurable() {
-                    public double measure(MetricConfig config, long now) {
-                        return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS);
-                    }
-                };
-            metrics.addMetric(new MetricName("last-heartbeat-seconds-ago",
-                this.metricGrpName,
-                "The number of seconds since the last controller heartbeat",
-                tags),
-                lastHeartbeat);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/DelayedTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/DelayedTask.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/DelayedTask.java
deleted file mode 100644
index 6cc07f7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/DelayedTask.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-
-package org.apache.kafka.copied.clients.consumer.internals;
-
-
-public interface DelayedTask {
-
-    /**
-     * Execute the task.
-     * @param now current time in milliseconds
-     */
-    void run(long now);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/DelayedTaskQueue.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/DelayedTaskQueue.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/DelayedTaskQueue.java
deleted file mode 100644
index dd02559..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/DelayedTaskQueue.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.kafka.copied.clients.consumer.internals;
-
-import java.util.Iterator;
-import java.util.PriorityQueue;
-
-/**
- * Tracks a set of tasks to be executed after a delay.
- */
-public class DelayedTaskQueue {
-
-    private PriorityQueue<Entry> tasks;
-
-    public DelayedTaskQueue() {
-        tasks = new PriorityQueue<Entry>();
-    }
-
-    /**
-     * Schedule a task for execution in the future.
-     *
-     * @param task the task to execute
-     * @param at the time at which to
-     */
-    public void add(DelayedTask task, long at) {
-        tasks.add(new Entry(task, at));
-    }
-
-    /**
-     * Remove a task from the queue if it is present
-     * @param task the task to be removed
-     * @returns true if a task was removed as a result of this call
-     */
-    public boolean remove(DelayedTask task) {
-        boolean wasRemoved = false;
-        Iterator<Entry> iterator = tasks.iterator();
-        while (iterator.hasNext()) {
-            Entry entry = iterator.next();
-            if (entry.task.equals(task)) {
-                iterator.remove();
-                wasRemoved = true;
-            }
-        }
-        return wasRemoved;
-    }
-
-    /**
-     * Get amount of time in milliseconds until the next event. Returns Long.MAX_VALUE if no tasks are scheduled.
-     *
-     * @return the remaining time in milliseconds
-     */
-    public long nextTimeout(long now) {
-        if (tasks.isEmpty())
-            return Long.MAX_VALUE;
-        else
-            return Math.max(tasks.peek().timeout - now, 0);
-    }
-
-    /**
-     * Run any ready tasks.
-     *
-     * @param now the current time
-     */
-    public void poll(long now) {
-        while (!tasks.isEmpty() && tasks.peek().timeout <= now) {
-            Entry entry = tasks.poll();
-            entry.task.run(now);
-        }
-    }
-
-    private static class Entry implements Comparable<Entry> {
-        DelayedTask task;
-        long timeout;
-
-        public Entry(DelayedTask task, long timeout) {
-            this.task = task;
-            this.timeout = timeout;
-        }
-
-        @Override
-        public int compareTo(Entry entry) {
-            return Long.compare(timeout, entry.timeout);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Fetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Fetcher.java
deleted file mode 100644
index 4767f3f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Fetcher.java
+++ /dev/null
@@ -1,498 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-
-package org.apache.kafka.copied.clients.consumer.internals;
-
-import org.apache.kafka.copied.clients.ClientResponse;
-import org.apache.kafka.copied.clients.Metadata;
-import org.apache.kafka.copied.clients.consumer.ConsumerRecord;
-import org.apache.kafka.copied.clients.consumer.NoOffsetForPartitionException;
-import org.apache.kafka.copied.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.copied.common.Cluster;
-import org.apache.kafka.copied.common.MetricName;
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.PartitionInfo;
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.errors.DisconnectException;
-import org.apache.kafka.copied.common.errors.InvalidMetadataException;
-import org.apache.kafka.copied.common.metrics.Metrics;
-import org.apache.kafka.copied.common.metrics.Sensor;
-import org.apache.kafka.copied.common.metrics.stats.Avg;
-import org.apache.kafka.copied.common.metrics.stats.Count;
-import org.apache.kafka.copied.common.metrics.stats.Max;
-import org.apache.kafka.copied.common.metrics.stats.Rate;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.record.LogEntry;
-import org.apache.kafka.copied.common.record.MemoryRecords;
-import org.apache.kafka.copied.common.requests.FetchRequest;
-import org.apache.kafka.copied.common.requests.FetchResponse;
-import org.apache.kafka.copied.common.requests.ListOffsetRequest;
-import org.apache.kafka.copied.common.requests.ListOffsetResponse;
-import org.apache.kafka.copied.common.serialization.Deserializer;
-import org.apache.kafka.copied.common.utils.Time;
-import org.apache.kafka.copied.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-
-/**
- * This class manage the fetching process with the brokers.
- */
-public class Fetcher<K, V> {
-    private static final long EARLIEST_OFFSET_TIMESTAMP = -2L;
-    private static final long LATEST_OFFSET_TIMESTAMP = -1L;
-
-    private static final Logger log = LoggerFactory.getLogger(Fetcher.class);
-
-    private final ConsumerNetworkClient client;
-    private final Time time;
-    private final int minBytes;
-    private final int maxWaitMs;
-    private final int fetchSize;
-    private final long retryBackoffMs;
-    private final boolean checkCrcs;
-    private final Metadata metadata;
-    private final FetchManagerMetrics sensors;
-    private final SubscriptionState subscriptions;
-    private final List<PartitionRecords<K, V>> records;
-    private final Deserializer<K> keyDeserializer;
-    private final Deserializer<V> valueDeserializer;
-
-    public Fetcher(ConsumerNetworkClient client,
-                   int minBytes,
-                   int maxWaitMs,
-                   int fetchSize,
-                   boolean checkCrcs,
-                   Deserializer<K> keyDeserializer,
-                   Deserializer<V> valueDeserializer,
-                   Metadata metadata,
-                   SubscriptionState subscriptions,
-                   Metrics metrics,
-                   String metricGrpPrefix,
-                   Map<String, String> metricTags,
-                   Time time,
-                   long retryBackoffMs) {
-
-        this.time = time;
-        this.client = client;
-        this.metadata = metadata;
-        this.subscriptions = subscriptions;
-        this.minBytes = minBytes;
-        this.maxWaitMs = maxWaitMs;
-        this.fetchSize = fetchSize;
-        this.checkCrcs = checkCrcs;
-
-        this.keyDeserializer = keyDeserializer;
-        this.valueDeserializer = valueDeserializer;
-
-        this.records = new LinkedList<PartitionRecords<K, V>>();
-
-        this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags);
-        this.retryBackoffMs = retryBackoffMs;
-    }
-
-    /**
-     * Set-up a fetch request for any node that we have assigned partitions for which doesn't have one.
-     *
-     * @param cluster The current cluster metadata
-     */
-    public void initFetches(Cluster cluster) {
-        for (Map.Entry<Node, FetchRequest> fetchEntry: createFetchRequests(cluster).entrySet()) {
-            final FetchRequest fetch = fetchEntry.getValue();
-            client.send(fetchEntry.getKey(), ApiKeys.FETCH, fetch)
-                    .addListener(new RequestFutureListener<ClientResponse>() {
-                        @Override
-                        public void onSuccess(ClientResponse response) {
-                            handleFetchResponse(response, fetch);
-                        }
-
-                        @Override
-                        public void onFailure(RuntimeException e) {
-                            log.debug("Fetch failed", e);
-                        }
-                    });
-        }
-    }
-
-    /**
-     * Update the fetch positions for the provided partitions.
-     * @param partitions
-     */
-    public void updateFetchPositions(Set<TopicPartition> partitions) {
-        // reset the fetch position to the committed position
-        for (TopicPartition tp : partitions) {
-            // skip if we already have a fetch position
-            if (subscriptions.fetched(tp) != null)
-                continue;
-
-            // TODO: If there are several offsets to reset, we could submit offset requests in parallel
-            if (subscriptions.isOffsetResetNeeded(tp)) {
-                resetOffset(tp);
-            } else if (subscriptions.committed(tp) == null) {
-                // there's no committed position, so we need to reset with the default strategy
-                subscriptions.needOffsetReset(tp);
-                resetOffset(tp);
-            } else {
-                log.debug("Resetting offset for partition {} to the committed offset {}",
-                        tp, subscriptions.committed(tp));
-                subscriptions.seek(tp, subscriptions.committed(tp));
-            }
-        }
-    }
-
-    /**
-     * Reset offsets for the given partition using the offset reset strategy.
-     *
-     * @param partition The given partition that needs reset offset
-     * @throws NoOffsetForPartitionException If no offset reset strategy is defined
-     */
-    private void resetOffset(TopicPartition partition) {
-        OffsetResetStrategy strategy = subscriptions.resetStrategy(partition);
-        final long timestamp;
-        if (strategy == OffsetResetStrategy.EARLIEST)
-            timestamp = EARLIEST_OFFSET_TIMESTAMP;
-        else if (strategy == OffsetResetStrategy.LATEST)
-            timestamp = LATEST_OFFSET_TIMESTAMP;
-        else
-            throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined");
-
-        log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase());
-        long offset = listOffset(partition, timestamp);
-        this.subscriptions.seek(partition, offset);
-    }
-
-    /**
-     * Fetch a single offset before the given timestamp for the partition.
-     *
-     * @param partition The partition that needs fetching offset.
-     * @param timestamp The timestamp for fetching offset.
-     * @return The offset of the message that is published before the given timestamp
-     */
-    private long listOffset(TopicPartition partition, long timestamp) {
-        while (true) {
-            RequestFuture<Long> future = sendListOffsetRequest(partition, timestamp);
-            client.poll(future);
-
-            if (future.succeeded())
-                return future.value();
-
-            if (!future.isRetriable())
-                throw future.exception();
-
-            if (future.exception() instanceof InvalidMetadataException)
-                client.awaitMetadataUpdate();
-            else
-                Utils.sleep(retryBackoffMs);
-        }
-    }
-
-    /**
-     * Return the fetched records, empty the record buffer and update the consumed position.
-     *
-     * @return The fetched records per partition
-     */
-    public Map<TopicPartition, List<ConsumerRecord<K, V>>> fetchedRecords() {
-        if (this.subscriptions.partitionAssignmentNeeded()) {
-            return Collections.emptyMap();
-        } else {
-            Map<TopicPartition, List<ConsumerRecord<K, V>>> drained = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
-            for (PartitionRecords<K, V> part : this.records) {
-                Long consumed = subscriptions.consumed(part.partition);
-                if (this.subscriptions.assignedPartitions().contains(part.partition)
-                    && (consumed == null || part.fetchOffset == consumed)) {
-                    List<ConsumerRecord<K, V>> records = drained.get(part.partition);
-                    if (records == null) {
-                        records = part.records;
-                        drained.put(part.partition, records);
-                    } else {
-                        records.addAll(part.records);
-                    }
-                    subscriptions.consumed(part.partition, part.records.get(part.records.size() - 1).offset() + 1);
-                } else {
-                    // these records aren't next in line based on the last consumed position, ignore them
-                    // they must be from an obsolete request
-                    log.debug("Ignoring fetched records for {} at offset {}", part.partition, part.fetchOffset);
-                }
-            }
-            this.records.clear();
-            return drained;
-        }
-    }
-
-    /**
-     * Fetch a single offset before the given timestamp for the partition.
-     *
-     * @param topicPartition The partition that needs fetching offset.
-     * @param timestamp The timestamp for fetching offset.
-     * @return A response which can be polled to obtain the corresponding offset.
-     */
-    private RequestFuture<Long> sendListOffsetRequest(final TopicPartition topicPartition, long timestamp) {
-        Map<TopicPartition, ListOffsetRequest.PartitionData> partitions = new HashMap<TopicPartition, ListOffsetRequest.PartitionData>(1);
-        partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1));
-        PartitionInfo info = metadata.fetch().partition(topicPartition);
-        if (info == null) {
-            metadata.add(topicPartition.topic());
-            log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition);
-            return RequestFuture.staleMetadata();
-        } else if (info.leader() == null) {
-            log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition);
-            return RequestFuture.leaderNotAvailable();
-        } else {
-            Node node = info.leader();
-            ListOffsetRequest request = new ListOffsetRequest(-1, partitions);
-            return client.send(node, ApiKeys.LIST_OFFSETS, request)
-                    .compose(new RequestFutureAdapter<ClientResponse, Long>() {
-                        @Override
-                        public void onSuccess(ClientResponse response, RequestFuture<Long> future) {
-                            handleListOffsetResponse(topicPartition, response, future);
-                        }
-                    });
-        }
-    }
-
-    /**
-     * Callback for the response of the list offset call above.
-     * @param topicPartition The partition that was fetched
-     * @param clientResponse The response from the server.
-     */
-    private void handleListOffsetResponse(TopicPartition topicPartition,
-                                          ClientResponse clientResponse,
-                                          RequestFuture<Long> future) {
-        if (clientResponse.wasDisconnected()) {
-            future.raise(new DisconnectException());
-        } else {
-            ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody());
-            short errorCode = lor.responseData().get(topicPartition).errorCode;
-            if (errorCode == Errors.NONE.code()) {
-                List<Long> offsets = lor.responseData().get(topicPartition).offsets;
-                if (offsets.size() != 1)
-                    throw new IllegalStateException("This should not happen.");
-                long offset = offsets.get(0);
-                log.debug("Fetched offset {} for partition {}", offset, topicPartition);
-
-                future.complete(offset);
-            } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code()
-                    || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) {
-                log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.",
-                        topicPartition);
-                future.raise(Errors.forCode(errorCode));
-            } else {
-                log.error("Attempt to fetch offsets for partition {} failed due to: {}",
-                        topicPartition, Errors.forCode(errorCode).exception().getMessage());
-                future.raise(new StaleMetadataException());
-            }
-        }
-    }
-
-    /**
-     * Create fetch requests for all nodes for which we have assigned partitions
-     * that have no existing requests in flight.
-     */
-    private Map<Node, FetchRequest> createFetchRequests(Cluster cluster) {
-        // create the fetch info
-        Map<Node, Map<TopicPartition, FetchRequest.PartitionData>> fetchable = new HashMap<Node, Map<TopicPartition, FetchRequest.PartitionData>>();
-        for (TopicPartition partition : subscriptions.assignedPartitions()) {
-            Node node = cluster.leaderFor(partition);
-            if (node == null) {
-                metadata.requestUpdate();
-            } else if (this.client.pendingRequestCount(node) == 0) {
-                // if there is a leader and no in-flight requests, issue a new fetch
-                Map<TopicPartition, FetchRequest.PartitionData> fetch = fetchable.get(node);
-                if (fetch == null) {
-                    fetch = new HashMap<TopicPartition, FetchRequest.PartitionData>();
-                    fetchable.put(node, fetch);
-                }
-                long offset = this.subscriptions.fetched(partition);
-                fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize));
-            }
-        }
-
-        // create the fetches
-        Map<Node, FetchRequest> requests = new HashMap<Node, FetchRequest>();
-        for (Map.Entry<Node, Map<TopicPartition, FetchRequest.PartitionData>> entry : fetchable.entrySet()) {
-            Node node = entry.getKey();
-            FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue());
-            requests.put(node, fetch);
-        }
-        return requests;
-    }
-
-    /**
-     * The callback for fetch completion
-     */
-    private void handleFetchResponse(ClientResponse resp, FetchRequest request) {
-        if (resp.wasDisconnected()) {
-            int correlation = resp.request().request().header().correlationId();
-            log.debug("Cancelled fetch request {} with correlation id {} due to node {} being disconnected",
-                resp.request(), correlation, resp.request().request().destination());
-        } else {
-            int totalBytes = 0;
-            int totalCount = 0;
-            FetchResponse response = new FetchResponse(resp.responseBody());
-            for (Map.Entry<TopicPartition, FetchResponse.PartitionData> entry : response.responseData().entrySet()) {
-                TopicPartition tp = entry.getKey();
-                FetchResponse.PartitionData partition = entry.getValue();
-                if (!subscriptions.assignedPartitions().contains(tp)) {
-                    log.debug("Ignoring fetched data for partition {} which is no longer assigned.", tp);
-                } else if (partition.errorCode == Errors.NONE.code()) {
-                    int bytes = 0;
-                    ByteBuffer buffer = partition.recordSet;
-                    MemoryRecords records = MemoryRecords.readableRecords(buffer);
-                    long fetchOffset = request.fetchData().get(tp).offset;
-                    List<ConsumerRecord<K, V>> parsed = new ArrayList<ConsumerRecord<K, V>>();
-                    for (LogEntry logEntry : records) {
-                        parsed.add(parseRecord(tp, logEntry));
-                        bytes += logEntry.size();
-                    }
-                    if (parsed.size() > 0) {
-                        ConsumerRecord<K, V> record = parsed.get(parsed.size() - 1);
-                        this.subscriptions.fetched(tp, record.offset() + 1);
-                        this.records.add(new PartitionRecords<K, V>(fetchOffset, tp, parsed));
-                        this.sensors.recordsFetchLag.record(partition.highWatermark - record.offset());
-                    }
-                    this.sensors.recordTopicFetchMetrics(tp.topic(), bytes, parsed.size());
-                    totalBytes += bytes;
-                    totalCount += parsed.size();
-                } else if (partition.errorCode == Errors.NOT_LEADER_FOR_PARTITION.code()
-                    || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) {
-                    this.metadata.requestUpdate();
-                } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) {
-                    // TODO: this could be optimized by grouping all out-of-range partitions
-                    log.info("Fetch offset {} is out of range, resetting offset", subscriptions.fetched(tp));
-                    subscriptions.needOffsetReset(tp);
-                } else if (partition.errorCode == Errors.UNKNOWN.code()) {
-                    log.warn("Unknown error fetching data for topic-partition {}", tp);
-                } else {
-                    throw new IllegalStateException("Unexpected error code " + partition.errorCode + " while fetching data");
-                }
-            }
-            this.sensors.bytesFetched.record(totalBytes);
-            this.sensors.recordsFetched.record(totalCount);
-        }
-        this.sensors.fetchLatency.record(resp.requestLatencyMs());
-    }
-
-    /**
-     * Parse the record entry, deserializing the key / value fields if necessary
-     */
-    private ConsumerRecord<K, V> parseRecord(TopicPartition partition, LogEntry logEntry) {
-        if (this.checkCrcs)
-            logEntry.record().ensureValid();
-
-        long offset = logEntry.offset();
-        ByteBuffer keyBytes = logEntry.record().key();
-        K key = keyBytes == null ? null : this.keyDeserializer.deserialize(partition.topic(), Utils.toArray(keyBytes));
-        ByteBuffer valueBytes = logEntry.record().value();
-        V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), Utils.toArray(valueBytes));
-
-        return new ConsumerRecord<K, V>(partition.topic(), partition.partition(), offset, key, value);
-    }
-
-    private static class PartitionRecords<K, V> {
-        public long fetchOffset;
-        public TopicPartition partition;
-        public List<ConsumerRecord<K, V>> records;
-
-        public PartitionRecords(long fetchOffset, TopicPartition partition, List<ConsumerRecord<K, V>> records) {
-            this.fetchOffset = fetchOffset;
-            this.partition = partition;
-            this.records = records;
-        }
-    }
-
-    private class FetchManagerMetrics {
-        public final Metrics metrics;
-        public final String metricGrpName;
-
-        public final Sensor bytesFetched;
-        public final Sensor recordsFetched;
-        public final Sensor fetchLatency;
-        public final Sensor recordsFetchLag;
-
-
-        public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
-            this.metrics = metrics;
-            this.metricGrpName = metricGrpPrefix + "-fetch-manager-metrics";
-
-            this.bytesFetched = metrics.sensor("bytes-fetched");
-            this.bytesFetched.add(new MetricName("fetch-size-avg",
-                this.metricGrpName,
-                "The average number of bytes fetched per request",
-                tags), new Avg());
-            this.bytesFetched.add(new MetricName("fetch-size-max",
-                this.metricGrpName,
-                "The maximum number of bytes fetched per request",
-                tags), new Max());
-            this.bytesFetched.add(new MetricName("bytes-consumed-rate",
-                this.metricGrpName,
-                "The average number of bytes consumed per second",
-                tags), new Rate());
-
-            this.recordsFetched = metrics.sensor("records-fetched");
-            this.recordsFetched.add(new MetricName("records-per-request-avg",
-                this.metricGrpName,
-                "The average number of records in each request",
-                tags), new Avg());
-            this.recordsFetched.add(new MetricName("records-consumed-rate",
-                this.metricGrpName,
-                "The average number of records consumed per second",
-                tags), new Rate());
-
-            this.fetchLatency = metrics.sensor("fetch-latency");
-            this.fetchLatency.add(new MetricName("fetch-latency-avg",
-                this.metricGrpName,
-                "The average time taken for a fetch request.",
-                tags), new Avg());
-            this.fetchLatency.add(new MetricName("fetch-latency-max",
-                this.metricGrpName,
-                "The max time taken for any fetch request.",
-                tags), new Max());
-            this.fetchLatency.add(new MetricName("fetch-rate",
-                this.metricGrpName,
-                "The number of fetch requests per second.",
-                tags), new Rate(new Count()));
-
-            this.recordsFetchLag = metrics.sensor("records-lag");
-            this.recordsFetchLag.add(new MetricName("records-lag-max",
-                this.metricGrpName,
-                "The maximum lag in terms of number of records for any partition in this window",
-                tags), new Max());
-        }
-
-        public void recordTopicFetchMetrics(String topic, int bytes, int records) {
-            // record bytes fetched
-            String name = "topic." + topic + ".bytes-fetched";
-            Sensor bytesFetched = this.metrics.getSensor(name);
-            if (bytesFetched == null)
-                bytesFetched = this.metrics.sensor(name);
-            bytesFetched.record(bytes);
-
-            // record records fetched
-            name = "topic." + topic + ".records-fetched";
-            Sensor recordsFetched = this.metrics.getSensor(name);
-            if (recordsFetched == null)
-                recordsFetched = this.metrics.sensor(name);
-            recordsFetched.record(records);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Heartbeat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Heartbeat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Heartbeat.java
deleted file mode 100644
index c5282a5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/Heartbeat.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer.internals;
-
-/**
- * A helper class for managing the heartbeat to the coordinator
- */
-public final class Heartbeat {
-    
-    /* The number of heartbeats to attempt to complete per session timeout interval.
-     * so, e.g., with a session timeout of 3 seconds we would attempt a heartbeat
-     * once per second.
-     */
-    public final static int HEARTBEATS_PER_SESSION_INTERVAL = 3;
-
-    private final long timeout;
-    private long lastHeartbeatSend;
-    private long lastHeartbeatReceive;
-    private long lastSessionReset;
-
-    public Heartbeat(long timeout, long now) {
-        this.timeout = timeout;
-        this.lastSessionReset = now;
-    }
-
-    public void sentHeartbeat(long now) {
-        this.lastHeartbeatSend = now;
-    }
-
-    public void receiveHeartbeat(long now) {
-        this.lastHeartbeatReceive = now;
-    }
-
-    public boolean shouldHeartbeat(long now) {
-        return timeToNextHeartbeat(now) == 0;
-    }
-    
-    public long lastHeartbeatSend() {
-        return this.lastHeartbeatSend;
-    }
-
-    public long timeToNextHeartbeat(long now) {
-        long timeSinceLastHeartbeat = now - Math.max(lastHeartbeatSend, lastSessionReset);
-
-        long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL;
-        if (timeSinceLastHeartbeat > hbInterval)
-            return 0;
-        else
-            return hbInterval - timeSinceLastHeartbeat;
-    }
-
-    public boolean sessionTimeoutExpired(long now) {
-        return now - Math.max(lastSessionReset, lastHeartbeatReceive) > timeout;
-    }
-
-    public long interval() {
-        return timeout / HEARTBEATS_PER_SESSION_INTERVAL;
-    }
-
-    public void resetSessionTimeout(long now) {
-        this.lastSessionReset = now;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/NoAvailableBrokersException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/NoAvailableBrokersException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/NoAvailableBrokersException.java
deleted file mode 100644
index 63ad46b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/NoAvailableBrokersException.java
+++ /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.
- */
-package org.apache.kafka.copied.clients.consumer.internals;
-
-import org.apache.kafka.copied.common.errors.InvalidMetadataException;
-
-/**
- * No brokers were available to complete a request.
- */
-public class NoAvailableBrokersException extends InvalidMetadataException {
-    private static final long serialVersionUID = 1L;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/NoOpConsumerRebalanceCallback.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
deleted file mode 100644
index d694dbf..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-
-package org.apache.kafka.copied.clients.consumer.internals;
-
-import org.apache.kafka.copied.clients.consumer.Consumer;
-import org.apache.kafka.copied.clients.consumer.ConsumerRebalanceCallback;
-import org.apache.kafka.copied.common.TopicPartition;
-
-import java.util.Collection;
-
-public class NoOpConsumerRebalanceCallback implements ConsumerRebalanceCallback {
-
-    @Override
-    public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {}
-
-    @Override
-    public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {}
-
-}


[12/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Crc32.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Crc32.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Crc32.java
new file mode 100644
index 0000000..4a23ed6
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Crc32.java
@@ -0,0 +1,396 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.utils;
+
+import java.util.zip.Checksum;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This class was taken from Hadoop org.apache.hadoop.util.PureJavaCrc32
+ * 
+ * A pure-java implementation of the CRC32 checksum that uses the same polynomial as the built-in native CRC32.
+ * 
+ * This is to avoid the JNI overhead for certain uses of Checksumming where many small pieces of data are checksummed in
+ * succession.
+ * 
+ * The current version is ~10x to 1.8x as fast as Sun's native java.util.zip.CRC32 in Java 1.6
+ * 
+ * @see java.util.zip.CRC32
+ */
+public class Crc32 implements Checksum {
+
+    /**
+     * Compute the CRC32 of the byte array
+     * 
+     * @param bytes The array to compute the checksum for
+     * @return The CRC32
+     */
+    public static long crc32(byte[] bytes) {
+        return crc32(bytes, 0, bytes.length);
+    }
+
+    /**
+     * Compute the CRC32 of the segment of the byte array given by the specified size and offset
+     * 
+     * @param bytes The bytes to checksum
+     * @param offset the offset at which to begin checksumming
+     * @param size the number of bytes to checksum
+     * @return The CRC32
+     */
+    public static long crc32(byte[] bytes, int offset, int size) {
+        Crc32 crc = new Crc32();
+        crc.update(bytes, offset, size);
+        return crc.getValue();
+    }
+
+    /** the current CRC value, bit-flipped */
+    private int crc;
+
+    /** Create a new PureJavaCrc32 object. */
+    public Crc32() {
+        reset();
+    }
+
+    @Override
+    public long getValue() {
+        return (~crc) & 0xffffffffL;
+    }
+
+    @Override
+    public void reset() {
+        crc = 0xffffffff;
+    }
+
+    @Override
+    public void update(byte[] b, int off, int len) {
+        int localCrc = crc;
+
+        while (len > 7) {
+            final int c0 = (b[off + 0] ^ localCrc) & 0xff;
+            final int c1 = (b[off + 1] ^ (localCrc >>>= 8)) & 0xff;
+            final int c2 = (b[off + 2] ^ (localCrc >>>= 8)) & 0xff;
+            final int c3 = (b[off + 3] ^ (localCrc >>>= 8)) & 0xff;
+            localCrc = (T[T8_7_START + c0] ^ T[T8_6_START + c1]) ^ (T[T8_5_START + c2] ^ T[T8_4_START + c3]);
+
+            final int c4 = b[off + 4] & 0xff;
+            final int c5 = b[off + 5] & 0xff;
+            final int c6 = b[off + 6] & 0xff;
+            final int c7 = b[off + 7] & 0xff;
+
+            localCrc ^= (T[T8_3_START + c4] ^ T[T8_2_START + c5]) ^ (T[T8_1_START + c6] ^ T[T8_0_START + c7]);
+
+            off += 8;
+            len -= 8;
+        }
+
+        /* loop unroll - duff's device style */
+        switch (len) {
+            case 7:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
+            case 6:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
+            case 5:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
+            case 4:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
+            case 3:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
+            case 2:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
+            case 1:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
+            default:
+                /* nothing */
+        }
+
+        // Publish crc out to object
+        crc = localCrc;
+    }
+
+    @Override
+    final public void update(int b) {
+        crc = (crc >>> 8) ^ T[T8_0_START + ((crc ^ b) & 0xff)];
+    }
+
+    /**
+     * Update the CRC32 given an integer
+     */
+    final public void updateInt(int input) {
+        update((byte) (input >> 24));
+        update((byte) (input >> 16));
+        update((byte) (input >> 8));
+        update((byte) input /* >> 0 */);
+    }
+
+    /*
+     * CRC-32 lookup tables generated by the polynomial 0xEDB88320. See also TestPureJavaCrc32.Table.
+     */
+    private static final int T8_0_START = 0 * 256;
+    private static final int T8_1_START = 1 * 256;
+    private static final int T8_2_START = 2 * 256;
+    private static final int T8_3_START = 3 * 256;
+    private static final int T8_4_START = 4 * 256;
+    private static final int T8_5_START = 5 * 256;
+    private static final int T8_6_START = 6 * 256;
+    private static final int T8_7_START = 7 * 256;
+
+    private static final int[] T = new int[] {
+        /* T8_0 */
+        0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, 0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, 0x0EDB8832,
+        0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, 0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, 0x1DB71064, 0x6AB020F2,
+        0xF3B97148, 0x84BE41DE, 0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, 0x136C9856, 0x646BA8C0, 0xFD62F97A,
+        0x8A65C9EC, 0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, 0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172,
+        0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, 0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, 0x32D86CE3,
+        0x45DF5C75, 0xDCD60DCF, 0xABD13D59, 0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, 0x21B4F4B5, 0x56B3C423,
+        0xCFBA9599, 0xB8BDA50F, 0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, 0x2F6F7C87, 0x58684C11, 0xC1611DAB,
+        0xB6662D3D, 0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, 0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433,
+        0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, 0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, 0x6B6B51F4,
+        0x1C6C6162, 0x856530D8, 0xF262004E, 0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, 0x65B0D9C6, 0x12B7E950,
+        0x8BBEB8EA, 0xFCB9887C, 0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, 0x4DB26158, 0x3AB551CE, 0xA3BC0074,
+        0xD4BB30E2, 0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, 0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0,
+        0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, 0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, 0x5768B525,
+        0x206F85B3, 0xB966D409, 0xCE61E49F, 0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, 0x59B33D17, 0x2EB40D81,
+        0xB7BD5C3B, 0xC0BA6CAD, 0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, 0xEAD54739, 0x9DD277AF, 0x04DB2615,
+        0x73DC1683, 0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, 0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1,
+        0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, 0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, 0xFED41B76,
+        0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, 0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, 0xD6D6A3E8, 0xA1D1937E,
+        0x38D8C2C4, 0x4FDFF252, 0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, 0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6,
+        0x41047A60, 0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, 0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236,
+        0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, 0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, 0xC2D7FFA7,
+        0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, 0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, 0x9C0906A9, 0xEB0E363F,
+        0x72076785, 0x05005713, 0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, 0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7,
+        0x0BDBDF21, 0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, 0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777,
+        0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, 0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, 0xA00AE278,
+        0xD70DD2EE, 0x4E048354, 0x3903B3C2, 0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, 0xAED16A4A, 0xD9D65ADC,
+        0x40DF0B66, 0x37D83BF0, 0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, 0xBDBDF21C, 0xCABAC28A, 0x53B39330,
+        0x24B4A3A6, 0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, 0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94,
+        0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D,
+        /* T8_1 */
+        0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, 0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, 0xC8D98A08,
+        0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, 0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, 0x4AC21251, 0x53D92310,
+        0x78F470D3, 0x61EF4192, 0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, 0x821B9859, 0x9B00A918, 0xB02DFADB,
+        0xA936CB9A, 0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, 0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761,
+        0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, 0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, 0x39316BAE,
+        0x202A5AEF, 0x0B07092C, 0x121C386D, 0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, 0xBB2AF3F7, 0xA231C2B6,
+        0x891C9175, 0x9007A034, 0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, 0x73F379FF, 0x6AE848BE, 0x41C51B7D,
+        0x58DE2A3C, 0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, 0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2,
+        0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, 0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, 0xBABB5D54,
+        0xA3A06C15, 0x888D3FD6, 0x91960E97, 0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, 0x7262D75C, 0x6B79E61D,
+        0x4054B5DE, 0x594F849F, 0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, 0x65FD6BA7, 0x7CE65AE6, 0x57CB0925,
+        0x4ED03864, 0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, 0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C,
+        0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, 0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, 0x4B53BCF2,
+        0x52488DB3, 0x7965DE70, 0x607EEF31, 0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, 0x838A36FA, 0x9A9107BB,
+        0xB1BC5478, 0xA8A76539, 0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, 0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD,
+        0x74C20E8C, 0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, 0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484,
+        0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, 0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, 0xB9980012,
+        0xA0833153, 0x8BAE6290, 0x92B553D1, 0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, 0xAE07BCE9, 0xB71C8DA8,
+        0x9C31DE6B, 0x852AEF2A, 0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, 0x66DE36E1, 0x7FC507A0, 0x54E85463,
+        0x4DF36522, 0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, 0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B,
+        0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, 0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, 0x4870E1B4,
+        0x516BD0F5, 0x7A468336, 0x635DB277, 0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, 0xAF96124A, 0xB68D230B,
+        0x9DA070C8, 0x84BB4189, 0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, 0x674F9842, 0x7E54A903, 0x5579FAC0,
+        0x4C62CB81, 0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, 0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8,
+        0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, 0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, 0x5E7EF3EC,
+        0x4765C2AD, 0x6C48916E, 0x7553A02F, 0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, 0x96A779E4, 0x8FBC48A5,
+        0xA4911B66, 0xBD8A2A27, 0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, 0x14BCE1BD, 0x0DA7D0FC, 0x268A833F,
+        0x3F91B27E, 0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, 0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876,
+        0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72,
+        /* T8_2 */
+        0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, 0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, 0x0E1351B8,
+        0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, 0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, 0x1C26A370, 0x1DE4C947,
+        0x1FA2771E, 0x1E601D29, 0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, 0x1235F2C8, 0x13F798FF, 0x11B126A6,
+        0x10734C91, 0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, 0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9,
+        0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, 0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, 0x3157BF84,
+        0x3095D5B3, 0x32D36BEA, 0x331101DD, 0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, 0x23624D4C, 0x22A0277B,
+        0x20E69922, 0x2124F315, 0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, 0x2D711CF4, 0x2CB376C3, 0x2EF5C89A,
+        0x2F37A2AD, 0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, 0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45,
+        0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, 0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, 0x6CBC2EB0,
+        0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, 0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, 0x62AF7F08, 0x636D153F,
+        0x612BAB66, 0x60E9C151, 0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, 0x48D7CB20, 0x4915A117, 0x4B531F4E,
+        0x4A917579, 0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, 0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1,
+        0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, 0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, 0x53F8C08C,
+        0x523AAABB, 0x507C14E2, 0x51BE7ED5, 0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, 0x5DEB9134, 0x5C29FB03,
+        0x5E6F455A, 0x5FAD2F6D, 0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, 0xE63CB35C, 0xE7FED96B, 0xE5B86732,
+        0xE47A0D05, 0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, 0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD,
+        0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, 0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, 0xF300E948,
+        0xF2C2837F, 0xF0843D26, 0xF1465711, 0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, 0xD9785D60, 0xD8BA3757,
+        0xDAFC890E, 0xDB3EE339, 0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, 0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6,
+        0xD52DB281, 0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, 0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049,
+        0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, 0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, 0xCC440774,
+        0xCD866D43, 0xCFC0D31A, 0xCE02B92D, 0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, 0x96A63E9C, 0x976454AB,
+        0x9522EAF2, 0x94E080C5, 0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, 0x98B56F24, 0x99770513, 0x9B31BB4A,
+        0x9AF3D17D, 0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, 0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5,
+        0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, 0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, 0xA9E2D0A0,
+        0xA820BA97, 0xAA6604CE, 0xABA46EF9, 0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, 0xA7F18118, 0xA633EB2F,
+        0xA4755576, 0xA5B73F41, 0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, 0xB5C473D0, 0xB40619E7, 0xB640A7BE,
+        0xB782CD89, 0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, 0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31,
+        0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED,
+        /* T8_3 */
+        0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, 0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, 0xC5B428EF,
+        0x7D084F8A, 0x6FBDE064, 0xD7018701, 0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, 0x5019579F, 0xE8A530FA,
+        0xFA109F14, 0x42ACF871, 0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, 0x95AD7F70, 0x2D111815, 0x3FA4B7FB,
+        0x8718D09E, 0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, 0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0,
+        0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, 0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, 0xEAE41086,
+        0x525877E3, 0x40EDD80D, 0xF851BF68, 0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, 0x7F496FF6, 0xC7F50893,
+        0xD540A77D, 0x6DFCC018, 0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, 0xBAFD4719, 0x0241207C, 0x10F48F92,
+        0xA848E8F7, 0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, 0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084,
+        0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, 0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, 0xCB0D0FA2,
+        0x73B168C7, 0x6104C729, 0xD9B8A04C, 0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, 0x0EB9274D, 0xB6054028,
+        0xA4B0EFC6, 0x1C0C88A3, 0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, 0x3B26F703, 0x839A9066, 0x912F3F88,
+        0x299358ED, 0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, 0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002,
+        0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, 0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, 0xE45D37CB,
+        0x5CE150AE, 0x4E54FF40, 0xF6E89825, 0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, 0x21E91F24, 0x99557841,
+        0x8BE0D7AF, 0x335CB0CA, 0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, 0x623B216C, 0xDA874609, 0xC832E9E7,
+        0x708E8E82, 0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, 0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D,
+        0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, 0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, 0x78F4C94B,
+        0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, 0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, 0x4D6B1905, 0xF5D77E60,
+        0xE762D18E, 0x5FDEB6EB, 0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, 0x88DF31EA, 0x3063568F, 0x22D6F961,
+        0x9A6A9E04, 0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, 0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174,
+        0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, 0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, 0x57A4F122,
+        0xEF189647, 0xFDAD39A9, 0x45115ECC, 0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, 0xF92F7951, 0x41931E34,
+        0x5326B1DA, 0xEB9AD6BF, 0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, 0x3C9B51BE, 0x842736DB, 0x96929935,
+        0x2E2EFE50, 0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, 0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120,
+        0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, 0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, 0xD67F4138,
+        0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, 0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, 0x13CB69D7, 0xAB770EB2,
+        0xB9C2A15C, 0x017EC639, 0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, 0x866616A7, 0x3EDA71C2, 0x2C6FDE2C,
+        0x94D3B949, 0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, 0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6,
+        0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1,
+        /* T8_4 */
+        0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, 0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, 0x30704BC1,
+        0x0D106271, 0x4AB018A1, 0x77D03111, 0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, 0x60E09782, 0x5D80BE32,
+        0x1A20C4E2, 0x2740ED52, 0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, 0x5090DC43, 0x6DF0F5F3, 0x2A508F23,
+        0x1730A693, 0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, 0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4,
+        0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, 0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, 0x0431C205,
+        0x3951EBB5, 0x7EF19165, 0x4391B8D5, 0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, 0x54A11E46, 0x69C137F6,
+        0x2E614D26, 0x13016496, 0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, 0x64D15587, 0x59B17C37, 0x1E1106E7,
+        0x23712F57, 0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, 0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459,
+        0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, 0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, 0x3813CFCB,
+        0x0573E67B, 0x42D39CAB, 0x7FB3B51B, 0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, 0x0863840A, 0x3503ADBA,
+        0x72A3D76A, 0x4FC3FEDA, 0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, 0x9932774D, 0xA4525EFD, 0xE3F2242D,
+        0xDE920D9D, 0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, 0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C,
+        0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, 0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, 0x0C52460F,
+        0x31326FBF, 0x7692156F, 0x4BF23CDF, 0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, 0x3C220DCE, 0x0142247E,
+        0x46E25EAE, 0x7B82771E, 0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, 0x44661652, 0x79063FE2, 0x3EA64532,
+        0x03C66C82, 0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, 0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743,
+        0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, 0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, 0xE1766CD1,
+        0xDC164561, 0x9BB63FB1, 0xA6D61601, 0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, 0x70279F96, 0x4D47B626,
+        0x0AE7CCF6, 0x3787E546, 0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, 0x4057D457, 0x7D37FDE7, 0x3A978737,
+        0x07F7AE87, 0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, 0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4,
+        0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, 0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, 0xD537E515,
+        0xE857CCA5, 0xAFF7B675, 0x92979FC5, 0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, 0x1C954E1B, 0x21F567AB,
+        0x66551D7B, 0x5B3534CB, 0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, 0x2CE505DA, 0x11852C6A, 0x562556BA,
+        0x6B457F0A, 0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, 0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349,
+        0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, 0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, 0x28D4C7DF,
+        0x15B4EE6F, 0x521494BF, 0x6F74BD0F, 0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, 0x18A48C1E, 0x25C4A5AE,
+        0x6264DF7E, 0x5F04F6CE, 0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, 0x4834505D, 0x755479ED, 0x32F4033D,
+        0x0F942A8D, 0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, 0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C,
+        0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C,
+        /* T8_5 */
+        0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, 0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, 0xEC53826D,
+        0x270F51C8, 0xA19B2366, 0x6AC7F0C3, 0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, 0x03D6029B, 0xC88AD13E,
+        0x4E1EA390, 0x85427035, 0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, 0xEF8580F6, 0x24D95353, 0xA24D21FD,
+        0x6911F258, 0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, 0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798,
+        0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, 0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, 0x706EC54D,
+        0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, 0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, 0x9FEB45BB, 0x54B7961E,
+        0xD223E4B0, 0x197F3715, 0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, 0x73B8C7D6, 0xB8E41473, 0x3E7066DD,
+        0xF52CB578, 0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, 0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4,
+        0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, 0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, 0x0C8E08F7,
+        0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, 0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, 0xE0DD8A9A, 0x2B81593F,
+        0xAD152B91, 0x6649F834, 0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, 0x08F40F5A, 0xC3A8DCFF, 0x453CAE51,
+        0x8E607DF4, 0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, 0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99,
+        0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, 0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, 0x90B34FD7,
+        0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, 0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, 0x7CE0CDBA, 0xB7BC1E1F,
+        0x31286CB1, 0xFA74BF14, 0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, 0x852156CE, 0x4E7D856B, 0xC8E9F7C5,
+        0x03B52460, 0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, 0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D,
+        0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, 0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, 0xF135942E,
+        0x3A69478B, 0xBCFD3525, 0x77A1E680, 0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, 0x191C11EE, 0xD240C24B,
+        0x54D4B0E5, 0x9F886340, 0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, 0xF54F9383, 0x3E134026, 0xB8873288,
+        0x73DBE12D, 0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, 0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB,
+        0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, 0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, 0x6D08D30E,
+        0xA65400AB, 0x20C07205, 0xEB9CA1A0, 0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, 0x8A795CA2, 0x41258F07,
+        0xC7B1FDA9, 0x0CED2E0C, 0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, 0x662ADECF, 0xAD760D6A, 0x2BE27FC4,
+        0xE0BEAC61, 0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, 0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97,
+        0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, 0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, 0x16441B82,
+        0xDD18C827, 0x5B8CBA89, 0x90D0692C, 0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, 0xFA1799EF, 0x314B4A4A,
+        0xB7DF38E4, 0x7C83EB41, 0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, 0x15921919, 0xDECECABC, 0x585AB812,
+        0x93066BB7, 0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, 0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA,
+        0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC,
+        /* T8_6 */
+        0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, 0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, 0x33EF4E67,
+        0x959845D3, 0xA4705F4E, 0x020754FA, 0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, 0x67DE9CCE, 0xC1A9977A,
+        0xF0418DE7, 0x56368653, 0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, 0x5431D2A9, 0xF246D91D, 0xC3AEC380,
+        0x65D9C834, 0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, 0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301,
+        0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, 0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, 0x081D53E8,
+        0xAE6A585C, 0x9F8242C1, 0x39F54975, 0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, 0x5C2C8141, 0xFA5B8AF5,
+        0xCBB39068, 0x6DC49BDC, 0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, 0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F,
+        0x5E2BD5BB, 0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, 0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7,
+        0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, 0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, 0x23D5E9B7,
+        0x85A2E203, 0xB44AF89E, 0x123DF32A, 0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, 0x103AA7D0, 0xB64DAC64,
+        0x87A5B6F9, 0x21D2BD4D, 0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, 0x8BB64CE5, 0x2DC14751, 0x1C295DCC,
+        0xBA5E5678, 0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, 0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F,
+        0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, 0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, 0x1827F438,
+        0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, 0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, 0x2BC8BA5F, 0x8DBFB1EB,
+        0xBC57AB76, 0x1A20A0C2, 0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, 0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8,
+        0x4DB1D47C, 0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, 0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B,
+        0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, 0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, 0xDC27385B,
+        0x7A5033EF, 0x4BB82972, 0xEDCF22C6, 0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, 0x47ABD36E, 0xE1DCD8DA,
+        0xD034C247, 0x7643C9F3, 0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, 0x74449D09, 0xD23396BD, 0xE3DB8C20,
+        0x45AC8794, 0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, 0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D,
+        0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, 0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, 0xE7D525D4,
+        0x41A22E60, 0x704A34FD, 0xD63D3F49, 0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, 0x3852BB98, 0x9E25B02C,
+        0xAFCDAAB1, 0x09BAA105, 0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, 0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6,
+        0x3A55EF62, 0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, 0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB,
+        0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, 0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, 0x03A0A617,
+        0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, 0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, 0x304FE870, 0x9638E3C4,
+        0xA7D0F959, 0x01A7F2ED, 0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, 0x647E3AD9, 0xC209316D, 0xF3E12BF0,
+        0x55962044, 0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, 0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23,
+        0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30,
+        /* T8_7 */
+        0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, 0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, 0xD3E51BB5,
+        0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, 0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, 0x7CBB312B, 0xB01131B5,
+        0x3E9E3656, 0xF23436C8, 0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, 0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3,
+        0x21D12D7D, 0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, 0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5,
+        0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, 0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, 0xAED97719,
+        0x62737787, 0xECFC7064, 0x205670FA, 0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, 0x01875D87, 0xCD2D5D19,
+        0x43A25AFA, 0x8F085A64, 0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, 0xD2624632, 0x1EC846AC, 0x9047414F,
+        0x5CED41D1, 0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, 0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4,
+        0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, 0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, 0x5526F3C6,
+        0x998CF358, 0x1703F4BB, 0xDBA9F425, 0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, 0x86C3E873, 0x4A69E8ED,
+        0xC4E6EF0E, 0x084CEF90, 0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, 0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6,
+        0x5E64A758, 0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, 0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED,
+        0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, 0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, 0x281A9F6A,
+        0xE4B09FF4, 0x6A3F9817, 0xA6959889, 0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, 0xFBFF84DF, 0x37558441,
+        0xB9DA83A2, 0x7570833C, 0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, 0xD7718B20, 0x1BDB8BBE, 0x95548C5D,
+        0x59FE8CC3, 0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, 0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776,
+        0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, 0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, 0xFC65AF44,
+        0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, 0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, 0xAA4DE78C, 0x66E7E712,
+        0xE868E0F1, 0x24C2E06F, 0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, 0x79A8FC39, 0xB502FCA7, 0x3B8DFB44,
+        0xF727FBDA, 0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, 0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144,
+        0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, 0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, 0x8159C3E8,
+        0x4DF3C376, 0xC37CC495, 0x0FD6C40B, 0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, 0xFEEC49CD, 0x32464953,
+        0xBCC94EB0, 0x70634E2E, 0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, 0x2D095278, 0xE1A352E6, 0x6F2C5505,
+        0xA386559B, 0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, 0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05,
+        0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, 0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, 0x83D02561,
+        0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, 0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, 0x50353ED4, 0x9C9F3E4A,
+        0x121039A9, 0xDEBA3937, 0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, 0xFF6B144A, 0x33C114D4, 0xBD4E1337,
+        0x71E413A9, 0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, 0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C,
+        0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6 };
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/KafkaThread.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/KafkaThread.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/KafkaThread.java
new file mode 100644
index 0000000..310c39b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/KafkaThread.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.kafka_backport.common.utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A wrapper for Thread that sets things up nicely
+ */
+public class KafkaThread extends Thread {
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    public KafkaThread(final String name, Runnable runnable, boolean daemon) {
+        super(runnable, name);
+        setDaemon(daemon);
+        setUncaughtExceptionHandler(new UncaughtExceptionHandler() {
+            public void uncaughtException(Thread t, Throwable e) {
+                log.error("Uncaught exception in " + name + ": ", e);
+            }
+        });
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/SystemTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/SystemTime.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/SystemTime.java
new file mode 100644
index 0000000..428258c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/SystemTime.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.kafka_backport.common.utils;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A time implementation that uses the system clock and sleep call
+ */
+public class SystemTime implements Time {
+
+    @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) {
+            // just wake up early
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Time.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Time.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Time.java
new file mode 100644
index 0000000..88bb1c0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Time.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.kafka_backport.common.utils;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * An interface abstracting the clock to use in unit testing classes that make use of clock time
+ */
+public interface Time {
+
+    /**
+     * The current time in milliseconds
+     */
+    public long milliseconds();
+
+    /**
+     * The current time in nanoseconds
+     */
+    public long nanoseconds();
+
+    /**
+     * Sleep for the given number of milliseconds
+     */
+    public void sleep(long ms);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Utils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Utils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Utils.java
new file mode 100755
index 0000000..e2a5463
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Utils.java
@@ -0,0 +1,506 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.utils;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Properties;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class Utils {
+
+    // This matches URIs of formats: host:port and protocol:\\host:port
+    // IPv6 is supported with [ip] pattern
+    private static final Pattern HOST_PORT_PATTERN = Pattern.compile(".*?\\[?([0-9a-z\\-.:]*)\\]?:([0-9]+)");
+
+    public static final String NL = System.getProperty("line.separator");
+
+    private static final Logger log = LoggerFactory.getLogger(Utils.class);
+
+    /**
+     * Turn the given UTF8 byte array into a string
+     * 
+     * @param bytes The byte array
+     * @return The string
+     */
+    public static String utf8(byte[] bytes) {
+        try {
+            return new String(bytes, "UTF8");
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException("This shouldn't happen.", e);
+        }
+    }
+
+    /**
+     * Turn a string into a utf8 byte[]
+     * 
+     * @param string The string
+     * @return The byte[]
+     */
+    public static byte[] utf8(String string) {
+        try {
+            return string.getBytes("UTF8");
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException("This shouldn't happen.", e);
+        }
+    }
+
+    /**
+     * Read an unsigned integer from the current position in the buffer, incrementing the position by 4 bytes
+     * 
+     * @param buffer The buffer to read from
+     * @return The integer read, as a long to avoid signedness
+     */
+    public static long readUnsignedInt(ByteBuffer buffer) {
+        return buffer.getInt() & 0xffffffffL;
+    }
+
+    /**
+     * Read an unsigned integer from the given position without modifying the buffers position
+     * 
+     * @param buffer the buffer to read from
+     * @param index the index from which to read the integer
+     * @return The integer read, as a long to avoid signedness
+     */
+    public static long readUnsignedInt(ByteBuffer buffer, int index) {
+        return buffer.getInt(index) & 0xffffffffL;
+    }
+
+    /**
+     * Read an unsigned integer stored in little-endian format from the {@link InputStream}.
+     * 
+     * @param in The stream to read from
+     * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS)
+     */
+    public static int readUnsignedIntLE(InputStream in) throws IOException {
+        return (in.read() << 8 * 0) 
+             | (in.read() << 8 * 1)
+             | (in.read() << 8 * 2)
+             | (in.read() << 8 * 3);
+    }
+
+    /**
+     * Read an unsigned integer stored in little-endian format from a byte array
+     * at a given offset.
+     * 
+     * @param buffer The byte array to read from
+     * @param offset The position in buffer to read from
+     * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS)
+     */
+    public static int readUnsignedIntLE(byte[] buffer, int offset) {
+        return (buffer[offset++] << 8 * 0)
+             | (buffer[offset++] << 8 * 1)
+             | (buffer[offset++] << 8 * 2)
+             | (buffer[offset]   << 8 * 3);
+    }
+
+    /**
+     * Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
+     * 
+     * @param buffer The buffer to write to
+     * @param value The value to write
+     */
+    public static void writetUnsignedInt(ByteBuffer buffer, long value) {
+        buffer.putInt((int) (value & 0xffffffffL));
+    }
+
+    /**
+     * Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
+     * 
+     * @param buffer The buffer to write to
+     * @param index The position in the buffer at which to begin writing
+     * @param value The value to write
+     */
+    public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) {
+        buffer.putInt(index, (int) (value & 0xffffffffL));
+    }
+
+    /**
+     * Write an unsigned integer in little-endian format to the {@link OutputStream}.
+     * 
+     * @param out The stream to write to
+     * @param value The value to write
+     */
+    public static void writeUnsignedIntLE(OutputStream out, int value) throws IOException {
+        out.write(value >>> 8 * 0);
+        out.write(value >>> 8 * 1);
+        out.write(value >>> 8 * 2);
+        out.write(value >>> 8 * 3);
+    }
+
+    /**
+     * Write an unsigned integer in little-endian format to a byte array
+     * at a given offset.
+     * 
+     * @param buffer The byte array to write to
+     * @param offset The position in buffer to write to
+     * @param value The value to write
+     */
+    public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) {
+        buffer[offset++] = (byte) (value >>> 8 * 0);
+        buffer[offset++] = (byte) (value >>> 8 * 1);
+        buffer[offset++] = (byte) (value >>> 8 * 2);
+        buffer[offset]   = (byte) (value >>> 8 * 3);
+    }
+
+
+    /**
+     * Get the absolute value of the given number. If the number is Int.MinValue return 0. This is different from
+     * java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!).
+     */
+    public static int abs(int n) {
+        return (n == Integer.MIN_VALUE) ? 0 : Math.abs(n);
+    }
+
+    /**
+     * Get the minimum of some long values.
+     * @param first Used to ensure at least one value
+     * @param rest The rest of longs to compare
+     * @return The minimum of all passed argument.
+     */
+    public static long min(long first, long ... rest) {
+        long min = first;
+        for (int i = 0; i < rest.length; i++) {
+            if (rest[i] < min)
+                min = rest[i];
+        }
+        return min;
+    }
+
+    /**
+     * Get the length for UTF8-encoding a string without encoding it first
+     * 
+     * @param s The string to calculate the length for
+     * @return The length when serialized
+     */
+    public static int utf8Length(CharSequence s) {
+        int count = 0;
+        for (int i = 0, len = s.length(); i < len; i++) {
+            char ch = s.charAt(i);
+            if (ch <= 0x7F) {
+                count++;
+            } else if (ch <= 0x7FF) {
+                count += 2;
+            } else if (Character.isHighSurrogate(ch)) {
+                count += 4;
+                ++i;
+            } else {
+                count += 3;
+            }
+        }
+        return count;
+    }
+
+    /**
+     * Read the given byte buffer into a byte array
+     */
+    public static byte[] toArray(ByteBuffer buffer) {
+        return toArray(buffer, 0, buffer.limit());
+    }
+
+    /**
+     * Read a byte array from the given offset and size in the buffer
+     */
+    public static byte[] toArray(ByteBuffer buffer, int offset, int size) {
+        byte[] dest = new byte[size];
+        if (buffer.hasArray()) {
+            System.arraycopy(buffer.array(), buffer.arrayOffset() + offset, dest, 0, size);
+        } else {
+            int pos = buffer.position();
+            buffer.get(dest);
+            buffer.position(pos);
+        }
+        return dest;
+    }
+
+    /**
+     * Check that the parameter t is not null
+     * 
+     * @param t The object to check
+     * @return t if it isn't null
+     * @throws NullPointerException if t is null.
+     */
+    public static <T> T notNull(T t) {
+        if (t == null)
+            throw new NullPointerException();
+        else
+            return t;
+    }
+
+    /**
+     * Sleep for a bit
+     * @param ms The duration of the sleep
+     */
+    public static void sleep(long ms) {
+        try {
+            Thread.sleep(ms);
+        } catch (InterruptedException e) {
+            // this is okay, we just wake up early
+        }
+    }
+
+    /**
+     * Instantiate the class
+     */
+    public static Object newInstance(Class<?> c) {
+        try {
+            return c.newInstance();
+        } catch (IllegalAccessException e) {
+            throw new KafkaException("Could not instantiate class " + c.getName(), e);
+        } catch (InstantiationException e) {
+            throw new KafkaException("Could not instantiate class " + c.getName() + " Does it have a public no-argument constructor?", e);
+        }
+    }
+
+    /**
+     * Generates 32 bit murmur2 hash from byte array
+     * @param data byte array to hash
+     * @return 32 bit hash of the given array
+     */
+    public static int murmur2(final byte[] data) {
+        int length = data.length;
+        int seed = 0x9747b28c;
+        // 'm' and 'r' are mixing constants generated offline.
+        // They're not really 'magic', they just happen to work well.
+        final int m = 0x5bd1e995;
+        final int r = 24;
+
+        // Initialize the hash to a random value
+        int h = seed ^ length;
+        int length4 = length / 4;
+
+        for (int i = 0; i < length4; i++) {
+            final int i4 = i * 4;
+            int k = (data[i4 + 0] & 0xff) + ((data[i4 + 1] & 0xff) << 8) + ((data[i4 + 2] & 0xff) << 16) + ((data[i4 + 3] & 0xff) << 24);
+            k *= m;
+            k ^= k >>> r;
+            k *= m;
+            h *= m;
+            h ^= k;
+        }
+
+        // Handle the last few bytes of the input array
+        switch (length % 4) {
+            case 3:
+                h ^= (data[(length & ~3) + 2] & 0xff) << 16;
+            case 2:
+                h ^= (data[(length & ~3) + 1] & 0xff) << 8;
+            case 1:
+                h ^= data[length & ~3] & 0xff;
+                h *= m;
+        }
+
+        h ^= h >>> 13;
+        h *= m;
+        h ^= h >>> 15;
+
+        return h;
+    }
+
+    /**
+     * Extracts the hostname from a "host:port" address string.
+     * @param address address string to parse
+     * @return hostname or null if the given address is incorrect
+     */
+    public static String getHost(String address) {
+        Matcher matcher = HOST_PORT_PATTERN.matcher(address);
+        return matcher.matches() ? matcher.group(1) : null;
+    }
+
+    /**
+     * Extracts the port number from a "host:port" address string.
+     * @param address address string to parse
+     * @return port number or null if the given address is incorrect
+     */
+    public static Integer getPort(String address) {
+        Matcher matcher = HOST_PORT_PATTERN.matcher(address);
+        return matcher.matches() ? Integer.parseInt(matcher.group(2)) : null;
+    }
+
+    /**
+     * Formats hostname and port number as a "host:port" address string,
+     * surrounding IPv6 addresses with braces '[', ']'
+     * @param host hostname
+     * @param port port number
+     * @return address string
+     */
+    public static String formatAddress(String host, Integer port) {
+        return host.contains(":")
+                ? "[" + host + "]:" + port // IPv6
+                : host + ":" + port;
+    }
+
+    /**
+     * Create a string representation of an array joined by the given separator
+     * @param strs The array of items
+     * @param seperator The separator
+     * @return The string representation.
+     */
+    public static <T> String join(T[] strs, String seperator) {
+        return join(Arrays.asList(strs), seperator);
+    }
+    
+    /**
+     * Create a string representation of a list joined by the given separator
+     * @param list The list of items
+     * @param seperator The separator
+     * @return The string representation.
+     */
+    public static <T> String join(Collection<T> list, String seperator) {
+        StringBuilder sb = new StringBuilder();
+        Iterator<T> iter = list.iterator();
+        while (iter.hasNext()) {
+            sb.append(iter.next());
+            if (iter.hasNext())
+                sb.append(seperator);  
+        }
+        return sb.toString();
+    }
+
+    /**
+     * Read a properties file from the given path
+     * @param filename The path of the file to read
+     */
+    public static Properties loadProps(String filename) throws IOException, FileNotFoundException {
+        Properties props = new Properties();
+        InputStream propStream = null;
+        try {
+            propStream = new FileInputStream(filename);
+            props.load(propStream);
+        } finally {
+            if (propStream != null)
+                propStream.close();
+        }
+        return props;
+    }
+
+    /**
+     * Get the stack trace from an exception as a string
+     */
+    public static String stackTrace(Throwable e) {
+        StringWriter sw = new StringWriter();
+        PrintWriter pw = new PrintWriter(sw);
+        e.printStackTrace(pw);
+        return sw.toString();
+    }
+
+    /**
+     * Create a new thread
+     * @param name The name of the thread
+     * @param runnable The work for the thread to do
+     * @param daemon Should the thread block JVM shutdown?
+     * @return The unstarted thread
+     */
+    public static Thread newThread(String name, Runnable runnable, Boolean daemon) {
+        Thread thread = new Thread(runnable, name);
+        thread.setDaemon(daemon);
+        thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+            public void uncaughtException(Thread t, Throwable e) {
+                log.error("Uncaught exception in thread '" + t.getName() + "':", e);
+            }
+        });
+        return thread;
+    }
+
+    /**
+     * Create a daemon thread
+     * @param name The name of the thread
+     * @param runnable The runnable to execute in the background
+     * @return The unstarted thread
+     */
+    public static Thread daemonThread(String name, Runnable runnable) {
+        return newThread(name, runnable, true);
+    }
+
+    /**
+     * Print an error message and shutdown the JVM
+     * @param message The error message
+     */
+    public static void croak(String message) {
+        System.err.println(message);
+        System.exit(1);
+    }
+
+    /**
+     * Read a buffer into a Byte array for the given offset and length
+     */
+    public static byte[] readBytes(ByteBuffer buffer, int offset, int length) {
+        byte[] dest = new byte[length];
+        if (buffer.hasArray()) {
+            System.arraycopy(buffer.array(), buffer.arrayOffset() + offset, dest, 0, length);
+        } else {
+            buffer.mark();
+            buffer.position(offset);
+            buffer.get(dest, 0, length);
+            buffer.reset();
+        }
+        return dest;
+    }
+
+    /**
+     * Read the given byte buffer into a Byte array
+     */
+    public static byte[] readBytes(ByteBuffer buffer) {
+        return Utils.readBytes(buffer, 0, buffer.limit());
+    }
+
+    /**
+     * Attempt to read a file as a string
+     * @throws IOException 
+     */
+    public static String readFileAsString(String path, Charset charset) throws IOException {
+        if (charset == null) charset = Charset.defaultCharset();
+        FileInputStream stream = new FileInputStream(new File(path));
+        String result = new String();
+        try {
+            FileChannel fc = stream.getChannel();
+            MappedByteBuffer bb = fc.map(FileChannel.MapMode.READ_ONLY, 0, fc.size());
+            result = charset.decode(bb).toString();
+        } finally {
+            stream.close();
+        }
+        return result;
+    }
+
+    public static String readFileAsString(String path) throws IOException {
+        return Utils.readFileAsString(path, Charset.defaultCharset());
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
new file mode 100644
index 0000000..5852c13
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
@@ -0,0 +1,10 @@
+/**
+ *  This package (and its sub-packages) contain only classes that arecopied from
+ *  the Apache Kafka project.
+ *
+ *  The package contains a "backport" of the new consumer API, in order to
+ *  give Flink access to its functionality until the API is properly released.
+ *
+ *  This is a temporary workaround!
+ */
+package org.apache.flink.kafka_backport;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java
index d5305a9..318b2f8 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/Fetcher.java
@@ -18,7 +18,7 @@ package org.apache.flink.streaming.connectors.internals;
 
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.kafka.copied.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.TopicPartition;
 
 import java.util.List;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/FlinkKafkaConsumerBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/FlinkKafkaConsumerBase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/FlinkKafkaConsumerBase.java
index 58c0e4a..7450410 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/FlinkKafkaConsumerBase.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/FlinkKafkaConsumerBase.java
@@ -30,11 +30,11 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.kafka.copied.clients.consumer.ConsumerConfig;
-import org.apache.kafka.copied.clients.consumer.KafkaConsumer;
-import org.apache.kafka.copied.common.PartitionInfo;
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.serialization.ByteArrayDeserializer;
+import org.apache.flink.kafka_backport.clients.consumer.ConsumerConfig;
+import org.apache.flink.kafka_backport.clients.consumer.KafkaConsumer;
+import org.apache.flink.kafka_backport.common.PartitionInfo;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.serialization.ByteArrayDeserializer;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/IncludedFetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/IncludedFetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/IncludedFetcher.java
index 90b989c..f1b9e60 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/IncludedFetcher.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/IncludedFetcher.java
@@ -18,12 +18,12 @@ package org.apache.flink.streaming.connectors.internals;
 
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.kafka.copied.clients.consumer.CommitType;
-import org.apache.kafka.copied.clients.consumer.ConsumerRecord;
-import org.apache.kafka.copied.clients.consumer.ConsumerRecords;
-import org.apache.kafka.copied.clients.consumer.KafkaConsumer;
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.serialization.ByteArrayDeserializer;
+import org.apache.flink.kafka_backport.clients.consumer.CommitType;
+import org.apache.flink.kafka_backport.clients.consumer.ConsumerRecord;
+import org.apache.flink.kafka_backport.clients.consumer.ConsumerRecords;
+import org.apache.flink.kafka_backport.clients.consumer.KafkaConsumer;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.serialization.ByteArrayDeserializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java
index 54c4e9f..1f244c1 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/streaming/connectors/internals/LegacyFetcher.java
@@ -31,9 +31,9 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.util.StringUtils;
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.PartitionInfo;
-import org.apache.kafka.copied.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.PartitionInfo;
+import org.apache.flink.kafka_backport.common.TopicPartition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientRequest.java
deleted file mode 100644
index 8231f2e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientRequest.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.kafka.copied.clients;
-
-import org.apache.kafka.copied.common.requests.RequestSend;
-
-/**
- * A request being sent to the server. This holds both the network send as well as the client-level metadata.
- */
-public final class ClientRequest {
-
-    private final long createdMs;
-    private final boolean expectResponse;
-    private final RequestSend request;
-    private final RequestCompletionHandler callback;
-
-    /**
-     * @param createdMs The unix timestamp in milliseconds for the time at which this request was created.
-     * @param expectResponse Should we expect a response message or is this request complete once it is sent?
-     * @param request The request
-     * @param callback A callback to execute when the response has been received (or null if no callback is necessary)
-     */
-    public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, RequestCompletionHandler callback) {
-        this.createdMs = createdMs;
-        this.callback = callback;
-        this.request = request;
-        this.expectResponse = expectResponse;
-    }
-
-    @Override
-    public String toString() {
-        return "ClientRequest(expectResponse=" + expectResponse + ", callback=" + callback + ", request=" + request
-                + ")";
-    }
-
-    public boolean expectResponse() {
-        return expectResponse;
-    }
-
-    public RequestSend request() {
-        return request;
-    }
-
-    public boolean hasCallback() {
-        return callback != null;
-    }
-
-    public RequestCompletionHandler callback() {
-        return callback;
-    }
-
-    public long createdTime() {
-        return createdMs;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientResponse.java
deleted file mode 100644
index 3f7393d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientResponse.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.kafka.copied.clients;
-
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-/**
- * A response from the server. Contains both the body of the response as well as the correlated request that was
- * originally sent.
- */
-public class ClientResponse {
-
-    private final long received;
-    private final boolean disconnected;
-    private final ClientRequest request;
-    private final Struct responseBody;
-
-    /**
-     * @param request The original request
-     * @param received The unix timestamp when this response was received
-     * @param disconnected Whether the client disconnected before fully reading a response
-     * @param responseBody The response contents (or null) if we disconnected or no response was expected
-     */
-    public ClientResponse(ClientRequest request, long received, boolean disconnected, Struct responseBody) {
-        super();
-        this.received = received;
-        this.disconnected = disconnected;
-        this.request = request;
-        this.responseBody = responseBody;
-    }
-
-    public long receivedTime() {
-        return received;
-    }
-
-    public boolean wasDisconnected() {
-        return disconnected;
-    }
-
-    public ClientRequest request() {
-        return request;
-    }
-
-    public Struct responseBody() {
-        return responseBody;
-    }
-
-    public boolean hasResponse() {
-        return responseBody != null;
-    }
-
-    public long requestLatencyMs() {
-        return receivedTime() - this.request.createdTime();
-    }
-
-    @Override
-    public String toString() {
-        return "ClientResponse(received=" + received +
-               ", disconnected=" +
-               disconnected +
-               ", request=" +
-               request +
-               ", responseBody=" +
-               responseBody +
-               ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientUtils.java
deleted file mode 100644
index d6743a3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClientUtils.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.kafka.copied.clients;
-
-import org.apache.kafka.copied.common.config.ConfigException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.kafka.copied.common.utils.Utils.getHost;
-import static org.apache.kafka.copied.common.utils.Utils.getPort;
-
-public class ClientUtils {
-    private static final Logger log = LoggerFactory.getLogger(ClientUtils.class);
-
-    public static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls) {
-        List<InetSocketAddress> addresses = new ArrayList<InetSocketAddress>();
-        for (String url : urls) {
-            if (url != null && url.length() > 0) {
-                String host = getHost(url);
-                Integer port = getPort(url);
-                if (host == null || port == null)
-                    throw new ConfigException("Invalid url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
-                try {
-                    InetSocketAddress address = new InetSocketAddress(host, port);
-                    if (address.isUnresolved())
-                        throw new ConfigException("DNS resolution failed for url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
-                    addresses.add(address);
-                } catch (NumberFormatException e) {
-                    throw new ConfigException("Invalid port in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
-                }
-            }
-        }
-        if (addresses.size() < 1)
-            throw new ConfigException("No bootstrap urls given in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
-        return addresses;
-    }
-
-    public static void closeQuietly(Closeable c, String name, AtomicReference<Throwable> firstException) {
-        if (c != null) {
-            try {
-                c.close();
-            } catch (Throwable t) {
-                firstException.compareAndSet(null, t);
-                log.error("Failed to close " + name, t);
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClusterConnectionStates.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClusterConnectionStates.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClusterConnectionStates.java
deleted file mode 100644
index e6ad253..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ClusterConnectionStates.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The state of our connection to each node in the cluster.
- * 
- */
-final class ClusterConnectionStates {
-    private final long reconnectBackoffMs;
-    private final Map<String, NodeConnectionState> nodeState;
-
-    public ClusterConnectionStates(long reconnectBackoffMs) {
-        this.reconnectBackoffMs = reconnectBackoffMs;
-        this.nodeState = new HashMap<String, NodeConnectionState>();
-    }
-
-    /**
-     * Return true iff we can currently initiate a new connection. This will be the case if we are not
-     * connected and haven't been connected for at least the minimum reconnection backoff period.
-     * @param id The connection id to check
-     * @param now The current time in MS
-     * @return true if we can initiate a new connection
-     */
-    public boolean canConnect(String id, long now) {
-        NodeConnectionState state = nodeState.get(id);
-        if (state == null)
-            return true;
-        else
-            return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs >= this.reconnectBackoffMs;
-    }
-
-    /**
-     * Return true if we are disconnected from the given node and can't re-establish a connection yet
-     * @param id The connection to check
-     * @param now The current time in ms
-     */
-    public boolean isBlackedOut(String id, long now) {
-        NodeConnectionState state = nodeState.get(id);
-        if (state == null)
-            return false;
-        else
-            return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs < this.reconnectBackoffMs;
-    }
-
-    /**
-     * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
-     * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
-     * connections.
-     * @param id The connection to check
-     * @param now The current time in ms
-     */
-    public long connectionDelay(String id, long now) {
-        NodeConnectionState state = nodeState.get(id);
-        if (state == null) return 0;
-        long timeWaited = now - state.lastConnectAttemptMs;
-        if (state.state == ConnectionState.DISCONNECTED) {
-            return Math.max(this.reconnectBackoffMs - timeWaited, 0);
-        } else {
-            // When connecting or connected, we should be able to delay indefinitely since other events (connection or
-            // data acked) will cause a wakeup once data can be sent.
-            return Long.MAX_VALUE;
-        }
-    }
-
-    /**
-     * Enter the connecting state for the given connection.
-     * @param id The id of the connection
-     * @param now The current time.
-     */
-    public void connecting(String id, long now) {
-        nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now));
-    }
-
-    /**
-     * Return true iff a specific connection is connected
-     * @param id The id of the connection to check
-     */
-    public boolean isConnected(String id) {
-        NodeConnectionState state = nodeState.get(id);
-        return state != null && state.state == ConnectionState.CONNECTED;
-    }
-
-    /**
-     * Return true iff we are in the process of connecting
-     * @param id The id of the connection
-     */
-    public boolean isConnecting(String id) {
-        NodeConnectionState state = nodeState.get(id);
-        return state != null && state.state == ConnectionState.CONNECTING;
-    }
-
-    /**
-     * Enter the connected state for the given connection
-     * @param id The connection identifier
-     */
-    public void connected(String id) {
-        NodeConnectionState nodeState = nodeState(id);
-        nodeState.state = ConnectionState.CONNECTED;
-    }
-
-    /**
-     * Enter the disconnected state for the given node
-     * @param id The connection we have disconnected
-     */
-    public void disconnected(String id) {
-        NodeConnectionState nodeState = nodeState(id);
-        nodeState.state = ConnectionState.DISCONNECTED;
-    }
-    
-    /**
-     * Get the state of a given connection
-     * @param id The id of the connection
-     * @return The state of our connection
-     */
-    public ConnectionState connectionState(String id) {
-        return nodeState(id).state;
-    }
-    
-    /**
-     * Get the state of a given node
-     * @param id The connection to fetch the state for
-     */
-    private NodeConnectionState nodeState(String id) {
-        NodeConnectionState state = this.nodeState.get(id);
-        if (state == null)
-            throw new IllegalStateException("No entry found for connection " + id);
-        return state;
-    }
-    
-    /**
-     * The state of our connection to a node
-     */
-    private static class NodeConnectionState {
-
-        ConnectionState state;
-        long lastConnectAttemptMs;
-
-        public NodeConnectionState(ConnectionState state, long lastConnectAttempt) {
-            this.state = state;
-            this.lastConnectAttemptMs = lastConnectAttempt;
-        }
-
-        public String toString() {
-            return "NodeState(" + state + ", " + lastConnectAttemptMs + ")";
-        }
-    }
-}
\ No newline at end of file


[07/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/ApiException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/ApiException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/ApiException.java
deleted file mode 100644
index 80edd9d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/ApiException.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.kafka.copied.common.errors;
-
-import org.apache.kafka.copied.common.KafkaException;
-
-/**
- * Any API exception that is part of the public protocol and should be a subclass of this class and be part of this
- * package.
- */
-public class ApiException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public ApiException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public ApiException(String message) {
-        super(message);
-    }
-
-    public ApiException(Throwable cause) {
-        super(cause);
-    }
-
-    public ApiException() {
-        super();
-    }
-
-    /* avoid the expensive and useless stack trace for api exceptions */
-    @Override
-    public Throwable fillInStackTrace() {
-        return this;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/ConsumerCoordinatorNotAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/ConsumerCoordinatorNotAvailableException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/ConsumerCoordinatorNotAvailableException.java
deleted file mode 100644
index 4830cd8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/ConsumerCoordinatorNotAvailableException.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.kafka.copied.common.errors;
-
-/**
- * The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has
- * not yet been created.
- */
-public class ConsumerCoordinatorNotAvailableException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public ConsumerCoordinatorNotAvailableException() {
-        super();
-    }
-
-    public ConsumerCoordinatorNotAvailableException(String message) {
-        super(message);
-    }
-
-    public ConsumerCoordinatorNotAvailableException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public ConsumerCoordinatorNotAvailableException(Throwable cause) {
-        super(cause);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/CorruptRecordException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/CorruptRecordException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/CorruptRecordException.java
deleted file mode 100644
index 00989a1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/CorruptRecordException.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.kafka.copied.common.errors;
-
-/**
- * This exception indicates a record has failed it's internal CRC check, this generally indicates network or disk
- * corruption.
- */
-public class CorruptRecordException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public CorruptRecordException() {
-        super("This message has failed it's CRC checksum or is otherwise corrupt.");
-    }
-
-    public CorruptRecordException(String message) {
-        super(message);
-    }
-
-    public CorruptRecordException(Throwable cause) {
-        super(cause);
-    }
-
-    public CorruptRecordException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/DisconnectException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/DisconnectException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/DisconnectException.java
deleted file mode 100644
index d044c79..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/DisconnectException.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.kafka.copied.common.errors;
-
-
-/**
- * Server disconnected before a request could be completed.
- */
-public class DisconnectException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public DisconnectException() {
-        super();
-    }
-
-    public DisconnectException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public DisconnectException(String message) {
-        super(message);
-    }
-
-    public DisconnectException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/IllegalGenerationException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/IllegalGenerationException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/IllegalGenerationException.java
deleted file mode 100644
index 0a6720b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/IllegalGenerationException.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.kafka.copied.common.errors;
-
-public class IllegalGenerationException extends RetriableException {
-    private static final long serialVersionUID = 1L;
-
-    public IllegalGenerationException() {
-        super();
-    }
-
-    public IllegalGenerationException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public IllegalGenerationException(String message) {
-        super(message);
-    }
-
-    public IllegalGenerationException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InterruptException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InterruptException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InterruptException.java
deleted file mode 100644
index 844de15..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InterruptException.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.kafka.copied.common.errors;
-
-import org.apache.kafka.copied.common.KafkaException;
-
-/**
- * An unchecked wrapper for InterruptedException
- */
-public class InterruptException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-    
-    public InterruptException(InterruptedException cause) {
-        super(cause);
-        Thread.currentThread().interrupt();
-    }
-    
-    public InterruptException(String message, InterruptedException cause) {
-        super(message, cause);
-        Thread.currentThread().interrupt();
-    }
-
-    public InterruptException(String message) {
-        super(message, new InterruptedException());
-        Thread.currentThread().interrupt();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidMetadataException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidMetadataException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidMetadataException.java
deleted file mode 100644
index 08943be..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidMetadataException.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.kafka.copied.common.errors;
-
-/**
- * An exception that may indicate the client's metadata is out of date
- */
-public abstract class InvalidMetadataException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public InvalidMetadataException() {
-        super();
-    }
-
-    public InvalidMetadataException(String message) {
-        super(message);
-    }
-
-    public InvalidMetadataException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public InvalidMetadataException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidRequiredAcksException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidRequiredAcksException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidRequiredAcksException.java
deleted file mode 100644
index 6345084..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidRequiredAcksException.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.kafka.copied.common.errors;
-
-public class InvalidRequiredAcksException extends ApiException {
-    private static final long serialVersionUID = 1L;
-
-    public InvalidRequiredAcksException(String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidTopicException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidTopicException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidTopicException.java
deleted file mode 100644
index bd4e09a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/InvalidTopicException.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.kafka.copied.common.errors;
-
-/**
- * The client has attempted to perform an operation on an invalid topic.
- */
-public class InvalidTopicException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public InvalidTopicException() {
-        super();
-    }
-
-    public InvalidTopicException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public InvalidTopicException(String message) {
-        super(message);
-    }
-
-    public InvalidTopicException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/LeaderNotAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/LeaderNotAvailableException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/LeaderNotAvailableException.java
deleted file mode 100644
index 12d631c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/LeaderNotAvailableException.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.errors;
-
-/**
- * There is no currently available leader for the given partition (either because a leadership election is in progress
- * or because all replicas are down).
- */
-public class LeaderNotAvailableException extends InvalidMetadataException {
-
-    private static final long serialVersionUID = 1L;
-
-    public LeaderNotAvailableException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NetworkException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NetworkException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NetworkException.java
deleted file mode 100644
index 73d6113..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NetworkException.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.kafka.copied.common.errors;
-
-/**
- * A misc. network-related IOException occurred when making a request. This could be because the client's metadata is
- * out of date and it is making a request to a node that is now dead.
- */
-public class NetworkException extends InvalidMetadataException {
-
-    private static final long serialVersionUID = 1L;
-
-    public NetworkException() {
-        super();
-    }
-
-    public NetworkException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public NetworkException(String message) {
-        super(message);
-    }
-
-    public NetworkException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotCoordinatorForConsumerException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotCoordinatorForConsumerException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotCoordinatorForConsumerException.java
deleted file mode 100644
index bd1d916..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotCoordinatorForConsumerException.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.kafka.copied.common.errors;
-
-/**
- * The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is
- * not a coordinator for.
- */
-public class NotCoordinatorForConsumerException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public NotCoordinatorForConsumerException() {
-        super();
-    }
-
-    public NotCoordinatorForConsumerException(String message) {
-        super(message);
-    }
-
-    public NotCoordinatorForConsumerException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public NotCoordinatorForConsumerException(Throwable cause) {
-        super(cause);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotEnoughReplicasAfterAppendException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotEnoughReplicasAfterAppendException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotEnoughReplicasAfterAppendException.java
deleted file mode 100644
index 3b7a32a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotEnoughReplicasAfterAppendException.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.errors;
-
-/**
- * Number of insync replicas for the partition is lower than min.insync.replicas This exception is raised when the low
- * ISR size is discovered *after* the message was already appended to the log. Producer retries will cause duplicates.
- */
-public class NotEnoughReplicasAfterAppendException extends RetriableException {
-    private static final long serialVersionUID = 1L;
-
-    public NotEnoughReplicasAfterAppendException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotEnoughReplicasException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotEnoughReplicasException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotEnoughReplicasException.java
deleted file mode 100644
index 406b336..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotEnoughReplicasException.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.kafka.copied.common.errors;
-
-/**
- * Number of insync replicas for the partition is lower than min.insync.replicas
- */
-public class NotEnoughReplicasException extends RetriableException {
-    private static final long serialVersionUID = 1L;
-
-    public NotEnoughReplicasException() {
-        super();
-    }
-
-    public NotEnoughReplicasException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public NotEnoughReplicasException(String message) {
-        super(message);
-    }
-
-    public NotEnoughReplicasException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotLeaderForPartitionException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotLeaderForPartitionException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotLeaderForPartitionException.java
deleted file mode 100644
index 993874a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/NotLeaderForPartitionException.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.kafka.copied.common.errors;
-
-/**
- * This server is not the leader for the given partition
- */
-public class NotLeaderForPartitionException extends InvalidMetadataException {
-
-    private static final long serialVersionUID = 1L;
-
-    public NotLeaderForPartitionException() {
-        super();
-    }
-
-    public NotLeaderForPartitionException(String message) {
-        super(message);
-    }
-
-    public NotLeaderForPartitionException(Throwable cause) {
-        super(cause);
-    }
-
-    public NotLeaderForPartitionException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetLoadInProgressException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetLoadInProgressException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetLoadInProgressException.java
deleted file mode 100644
index 09cf02f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetLoadInProgressException.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.kafka.copied.common.errors;
-
-/**
- * The broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change
- * for that offsets topic partition).
- */
-public class OffsetLoadInProgressException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public OffsetLoadInProgressException() {
-        super();
-    }
-
-    public OffsetLoadInProgressException(String message) {
-        super(message);
-    }
-
-    public OffsetLoadInProgressException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public OffsetLoadInProgressException(Throwable cause) {
-        super(cause);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetMetadataTooLarge.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetMetadataTooLarge.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetMetadataTooLarge.java
deleted file mode 100644
index c840318..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetMetadataTooLarge.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.kafka.copied.common.errors;
-
-/**
- * The client has tried to save its offset with associated metadata larger than the maximum size allowed by the server.
- */
-public class OffsetMetadataTooLarge extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public OffsetMetadataTooLarge() {
-    }
-
-    public OffsetMetadataTooLarge(String message) {
-        super(message);
-    }
-
-    public OffsetMetadataTooLarge(Throwable cause) {
-        super(cause);
-    }
-
-    public OffsetMetadataTooLarge(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetOutOfRangeException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetOutOfRangeException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetOutOfRangeException.java
deleted file mode 100644
index df3a867..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/OffsetOutOfRangeException.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.kafka.copied.common.errors;
-
-/**
- * This offset is either larger or smaller than the range of offsets the server has for the given partition.
- * 
- */
-public class OffsetOutOfRangeException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public OffsetOutOfRangeException() {
-    }
-
-    public OffsetOutOfRangeException(String message) {
-        super(message);
-    }
-
-    public OffsetOutOfRangeException(Throwable cause) {
-        super(cause);
-    }
-
-    public OffsetOutOfRangeException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RecordBatchTooLargeException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RecordBatchTooLargeException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RecordBatchTooLargeException.java
deleted file mode 100644
index 009600a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RecordBatchTooLargeException.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.kafka.copied.common.errors;
-
-/**
- * This record batch is larger than the maximum allowable size
- */
-public class RecordBatchTooLargeException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public RecordBatchTooLargeException() {
-        super();
-    }
-
-    public RecordBatchTooLargeException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public RecordBatchTooLargeException(String message) {
-        super(message);
-    }
-
-    public RecordBatchTooLargeException(Throwable cause) {
-        super(cause);
-    }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RecordTooLargeException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RecordTooLargeException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RecordTooLargeException.java
deleted file mode 100644
index e699661..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RecordTooLargeException.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.kafka.copied.common.errors;
-
-/**
- * This record is larger than the maximum allowable size
- */
-public class RecordTooLargeException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public RecordTooLargeException() {
-        super();
-    }
-
-    public RecordTooLargeException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public RecordTooLargeException(String message) {
-        super(message);
-    }
-
-    public RecordTooLargeException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RetriableException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RetriableException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RetriableException.java
deleted file mode 100644
index c0fc82d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/RetriableException.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.kafka.copied.common.errors;
-
-/**
- * A retryable exception is a transient exception that if retried may succeed.
- */
-public abstract class RetriableException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public RetriableException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public RetriableException(String message) {
-        super(message);
-    }
-
-    public RetriableException(Throwable cause) {
-        super(cause);
-    }
-
-    public RetriableException() {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/SerializationException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/SerializationException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/SerializationException.java
deleted file mode 100644
index b44b0c2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/SerializationException.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.kafka.copied.common.errors;
-
-import org.apache.kafka.copied.common.KafkaException;
-
-/**
- *  Any exception during serialization in the producer
- */
-public class SerializationException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public SerializationException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public SerializationException(String message) {
-        super(message);
-    }
-
-    public SerializationException(Throwable cause) {
-        super(cause);
-    }
-
-    public SerializationException() {
-        super();
-    }
-
-    /* avoid the expensive and useless stack trace for serialization exceptions */
-    @Override
-    public Throwable fillInStackTrace() {
-        return this;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/TimeoutException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/TimeoutException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/TimeoutException.java
deleted file mode 100644
index a18b902..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/TimeoutException.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.kafka.copied.common.errors;
-
-/**
- * Indicates that a request timed out.
- */
-public class TimeoutException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public TimeoutException() {
-        super();
-    }
-
-    public TimeoutException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public TimeoutException(String message) {
-        super(message);
-    }
-
-    public TimeoutException(Throwable cause) {
-        super(cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownConsumerIdException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownConsumerIdException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownConsumerIdException.java
deleted file mode 100644
index 90d3f58..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownConsumerIdException.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.kafka.copied.common.errors;
-
-public class UnknownConsumerIdException extends RetriableException {
-    private static final long serialVersionUID = 1L;
-
-    public UnknownConsumerIdException() {
-        super();
-    }
-
-    public UnknownConsumerIdException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public UnknownConsumerIdException(String message) {
-        super(message);
-    }
-
-    public UnknownConsumerIdException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownServerException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownServerException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownServerException.java
deleted file mode 100644
index 3aad11c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownServerException.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.kafka.copied.common.errors;
-
-/**
- * An error occurred on the server for which the client doesn't have a corresponding error code. This is generally an
- * unexpected error.
- * 
- */
-public class UnknownServerException extends ApiException {
-
-    private static final long serialVersionUID = 1L;
-
-    public UnknownServerException() {
-    }
-
-    public UnknownServerException(String message) {
-        super(message);
-    }
-
-    public UnknownServerException(Throwable cause) {
-        super(cause);
-    }
-
-    public UnknownServerException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownTopicOrPartitionException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownTopicOrPartitionException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownTopicOrPartitionException.java
deleted file mode 100644
index 92bfaf5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/errors/UnknownTopicOrPartitionException.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.kafka.copied.common.errors;
-
-/**
- * This topic/partition doesn't exist
- */
-public class UnknownTopicOrPartitionException extends InvalidMetadataException {
-
-    private static final long serialVersionUID = 1L;
-
-    public UnknownTopicOrPartitionException() {
-    }
-
-    public UnknownTopicOrPartitionException(String message) {
-        super(message);
-    }
-
-    public UnknownTopicOrPartitionException(Throwable throwable) {
-        super(throwable);
-    }
-
-    public UnknownTopicOrPartitionException(String message, Throwable throwable) {
-        super(message, throwable);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/CompoundStat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/CompoundStat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/CompoundStat.java
deleted file mode 100644
index 9e7b0be..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/CompoundStat.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.kafka.copied.common.metrics;
-
-import org.apache.kafka.copied.common.MetricName;
-
-import java.util.List;
-
-/**
- * A compound stat is a stat where a single measurement and associated data structure feeds many metrics. This is the
- * example for a histogram which has many associated percentiles.
- */
-public interface CompoundStat extends Stat {
-
-    public List<NamedMeasurable> stats();
-
-    public static class NamedMeasurable {
-
-        private final MetricName name;
-        private final Measurable stat;
-
-        public NamedMeasurable(MetricName name, Measurable stat) {
-            super();
-            this.name = name;
-            this.stat = stat;
-        }
-
-        public MetricName name() {
-            return name;
-        }
-
-        public Measurable stat() {
-            return stat;
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/JmxReporter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/JmxReporter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/JmxReporter.java
deleted file mode 100644
index c160c61..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/JmxReporter.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.metrics;
-
-import org.apache.kafka.copied.common.KafkaException;
-import org.apache.kafka.copied.common.MetricName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.management.Attribute;
-import javax.management.AttributeList;
-import javax.management.AttributeNotFoundException;
-import javax.management.DynamicMBean;
-import javax.management.InvalidAttributeValueException;
-import javax.management.JMException;
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanException;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.ReflectionException;
-import java.lang.management.ManagementFactory;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Register metrics in JMX as dynamic mbeans based on the metric names
- */
-public class JmxReporter implements MetricsReporter {
-
-    private static final Logger log = LoggerFactory.getLogger(JmxReporter.class);
-    private static final Object LOCK = new Object();
-    private String prefix;
-    private final Map<String, KafkaMbean> mbeans = new HashMap<String, KafkaMbean>();
-
-    public JmxReporter() {
-        this("");
-    }
-
-    /**
-     * Create a JMX reporter that prefixes all metrics with the given string.
-     */
-    public JmxReporter(String prefix) {
-        this.prefix = prefix;
-    }
-
-    @Override
-    public void configure(Map<String, ?> configs) {}
-
-    @Override
-    public void init(List<KafkaMetric> metrics) {
-        synchronized (LOCK) {
-            for (KafkaMetric metric : metrics)
-                addAttribute(metric);
-            for (KafkaMbean mbean : mbeans.values())
-                reregister(mbean);
-        }
-    }
-
-    @Override
-    public void metricChange(KafkaMetric metric) {
-        synchronized (LOCK) {
-            KafkaMbean mbean = addAttribute(metric);
-            reregister(mbean);
-        }
-    }
-
-    private KafkaMbean addAttribute(KafkaMetric metric) {
-        try {
-            MetricName metricName = metric.metricName();
-            String mBeanName = getMBeanName(metricName);
-            if (!this.mbeans.containsKey(mBeanName))
-                mbeans.put(mBeanName, new KafkaMbean(mBeanName));
-            KafkaMbean mbean = this.mbeans.get(mBeanName);
-            mbean.setAttribute(metricName.name(), metric);
-            return mbean;
-        } catch (JMException e) {
-            throw new KafkaException("Error creating mbean attribute for metricName :" + metric.metricName(), e);
-        }
-    }
-
-    /**
-     * @param metricName
-     * @return standard JMX MBean name in the following format domainName:type=metricType,key1=val1,key2=val2
-     */
-    private String getMBeanName(MetricName metricName) {
-        StringBuilder mBeanName = new StringBuilder();
-        mBeanName.append(prefix);
-        mBeanName.append(":type=");
-        mBeanName.append(metricName.group());
-        for (Map.Entry<String, String> entry : metricName.tags().entrySet()) {
-            if (entry.getKey().length() <= 0 || entry.getValue().length() <= 0)
-                continue;
-            mBeanName.append(",");
-            mBeanName.append(entry.getKey());
-            mBeanName.append("=");
-            mBeanName.append(entry.getValue());
-        }
-        return mBeanName.toString();
-    }
-
-    public void close() {
-        synchronized (LOCK) {
-            for (KafkaMbean mbean : this.mbeans.values())
-                unregister(mbean);
-        }
-    }
-
-    private void unregister(KafkaMbean mbean) {
-        MBeanServer server = ManagementFactory.getPlatformMBeanServer();
-        try {
-            if (server.isRegistered(mbean.name()))
-                server.unregisterMBean(mbean.name());
-        } catch (JMException e) {
-            throw new KafkaException("Error unregistering mbean", e);
-        }
-    }
-
-    private void reregister(KafkaMbean mbean) {
-        unregister(mbean);
-        try {
-            ManagementFactory.getPlatformMBeanServer().registerMBean(mbean, mbean.name());
-        } catch (JMException e) {
-            throw new KafkaException("Error registering mbean " + mbean.name(), e);
-        }
-    }
-
-    private static class KafkaMbean implements DynamicMBean {
-        private final ObjectName objectName;
-        private final Map<String, KafkaMetric> metrics;
-
-        public KafkaMbean(String mbeanName) throws MalformedObjectNameException {
-            this.metrics = new HashMap<String, KafkaMetric>();
-            this.objectName = new ObjectName(mbeanName);
-        }
-
-        public ObjectName name() {
-            return objectName;
-        }
-
-        public void setAttribute(String name, KafkaMetric metric) {
-            this.metrics.put(name, metric);
-        }
-
-        @Override
-        public Object getAttribute(String name) throws AttributeNotFoundException, MBeanException, ReflectionException {
-            if (this.metrics.containsKey(name))
-                return this.metrics.get(name).value();
-            else
-                throw new AttributeNotFoundException("Could not find attribute " + name);
-        }
-
-        @Override
-        public AttributeList getAttributes(String[] names) {
-            try {
-                AttributeList list = new AttributeList();
-                for (String name : names)
-                    list.add(new Attribute(name, getAttribute(name)));
-                return list;
-            } catch (Exception e) {
-                log.error("Error getting JMX attribute: ", e);
-                return new AttributeList();
-            }
-        }
-
-        @Override
-        public MBeanInfo getMBeanInfo() {
-            MBeanAttributeInfo[] attrs = new MBeanAttributeInfo[metrics.size()];
-            int i = 0;
-            for (Map.Entry<String, KafkaMetric> entry : this.metrics.entrySet()) {
-                String attribute = entry.getKey();
-                KafkaMetric metric = entry.getValue();
-                attrs[i] = new MBeanAttributeInfo(attribute,
-                                                  double.class.getName(),
-                                                  metric.metricName().description(),
-                                                  true,
-                                                  false,
-                                                  false);
-                i += 1;
-            }
-            return new MBeanInfo(this.getClass().getName(), "", attrs, null, null, null);
-        }
-
-        @Override
-        public Object invoke(String name, Object[] params, String[] sig) throws MBeanException, ReflectionException {
-            throw new UnsupportedOperationException("Set not allowed.");
-        }
-
-        @Override
-        public void setAttribute(Attribute attribute) throws AttributeNotFoundException,
-                InvalidAttributeValueException,
-                MBeanException,
-                ReflectionException {
-            throw new UnsupportedOperationException("Set not allowed.");
-        }
-
-        @Override
-        public AttributeList setAttributes(AttributeList list) {
-            throw new UnsupportedOperationException("Set not allowed.");
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/KafkaMetric.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/KafkaMetric.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/KafkaMetric.java
deleted file mode 100644
index 0a7296c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/KafkaMetric.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.kafka.copied.common.metrics;
-
-import org.apache.kafka.copied.common.Metric;
-import org.apache.kafka.copied.common.MetricName;
-import org.apache.kafka.copied.common.utils.Time;
-
-public final class KafkaMetric implements Metric {
-
-    private MetricName metricName;
-    private final Object lock;
-    private final Time time;
-    private final Measurable measurable;
-    private MetricConfig config;
-
-    KafkaMetric(Object lock, MetricName metricName, Measurable measurable, MetricConfig config, Time time) {
-        super();
-        this.metricName = metricName;
-        this.lock = lock;
-        this.measurable = measurable;
-        this.config = config;
-        this.time = time;
-    }
-
-    MetricConfig config() {
-        return this.config;
-    }
-
-    @Override
-    public MetricName metricName() {
-        return this.metricName;
-    }
-
-    @Override
-    public double value() {
-        synchronized (this.lock) {
-            return value(time.milliseconds());
-        }
-    }
-
-    double value(long timeMs) {
-        return this.measurable.measure(config, timeMs);
-    }
-
-    public void config(MetricConfig config) {
-        synchronized (lock) {
-            this.config = config;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Measurable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Measurable.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Measurable.java
deleted file mode 100644
index 22bff57..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/Measurable.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.kafka.copied.common.metrics;
-
-/**
- * A measurable quantity that can be registered as a metric
- */
-public interface Measurable {
-
-    /**
-     * Measure this quantity and return the result as a double
-     * @param config The configuration for this metric
-     * @param now The POSIX time in milliseconds the measurement is being taken
-     * @return The measured value
-     */
-    public double measure(MetricConfig config, long now);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MeasurableStat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MeasurableStat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MeasurableStat.java
deleted file mode 100644
index 1856543..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MeasurableStat.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.metrics;
-
-import org.apache.kafka.copied.common.metrics.stats.Avg;
-import org.apache.kafka.copied.common.metrics.stats.Count;
-import org.apache.kafka.copied.common.metrics.stats.Max;
-
-/**
- * A MeasurableStat is a {@link Stat} that is also {@link Measurable} (i.e. can produce a single floating point value).
- * This is the interface used for most of the simple statistics such as {@link Avg},
- * {@link Max}, {@link Count}, etc.
- */
-public interface MeasurableStat extends Stat, Measurable {
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MetricConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MetricConfig.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MetricConfig.java
deleted file mode 100644
index 09b61cd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/metrics/MetricConfig.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.metrics;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Configuration values for metrics
- */
-public class MetricConfig {
-
-    private Quota quota;
-    private int samples;
-    private long eventWindow;
-    private long timeWindowMs;
-    private TimeUnit unit;
-
-    public MetricConfig() {
-        super();
-        this.quota = null;
-        this.samples = 2;
-        this.eventWindow = Long.MAX_VALUE;
-        this.timeWindowMs = TimeUnit.MILLISECONDS.convert(30, TimeUnit.SECONDS);
-        this.unit = TimeUnit.SECONDS;
-    }
-
-    public Quota quota() {
-        return this.quota;
-    }
-
-    public MetricConfig quota(Quota quota) {
-        this.quota = quota;
-        return this;
-    }
-
-    public long eventWindow() {
-        return eventWindow;
-    }
-
-    public MetricConfig eventWindow(long window) {
-        this.eventWindow = window;
-        return this;
-    }
-
-    public long timeWindowMs() {
-        return timeWindowMs;
-    }
-
-    public MetricConfig timeWindow(long window, TimeUnit unit) {
-        this.timeWindowMs = TimeUnit.MILLISECONDS.convert(window, unit);
-        return this;
-    }
-
-    public int samples() {
-        return this.samples;
-    }
-
-    public MetricConfig samples(int samples) {
-        if (samples < 1)
-            throw new IllegalArgumentException("The number of samples must be at least 1.");
-        this.samples = samples;
-        return this;
-    }
-
-    public TimeUnit timeUnit() {
-        return unit;
-    }
-
-    public MetricConfig timeUnit(TimeUnit unit) {
-        this.unit = unit;
-        return this;
-    }
-}


[49/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/NetworkClient.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/NetworkClient.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/NetworkClient.java
deleted file mode 100644
index 90258bb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/NetworkClient.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.kafka_backport.clients;
-
-import org.apache.flink.kafka_backport.common.Node;
-import org.apache.flink.kafka_backport.common.network.NetworkReceive;
-import org.apache.flink.kafka_backport.common.network.Send;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.requests.MetadataRequest;
-import org.apache.flink.kafka_backport.common.requests.MetadataResponse;
-import org.apache.flink.kafka_backport.common.requests.RequestHeader;
-import org.apache.flink.kafka_backport.common.requests.RequestSend;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-import org.apache.flink.kafka_backport.common.Cluster;
-import org.apache.flink.kafka_backport.common.network.Selectable;
-import org.apache.flink.kafka_backport.common.requests.ResponseHeader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A network client for asynchronous request/response network i/o. This is an internal class used to implement the
- * user-facing producer and consumer clients.
- * <p>
- * This class is not thread-safe!
- */
-public class NetworkClient implements KafkaClient {
-
-    private static final Logger log = LoggerFactory.getLogger(NetworkClient.class);
-
-    /* the selector used to perform network i/o */
-    private final Selectable selector;
-
-    /* the current cluster metadata */
-    private final Metadata metadata;
-
-    /* the state of each node's connection */
-    private final ClusterConnectionStates connectionStates;
-
-    /* the set of requests currently being sent or awaiting a response */
-    private final InFlightRequests inFlightRequests;
-
-    /* the socket send buffer size in bytes */
-    private final int socketSendBuffer;
-
-    /* the socket receive size buffer in bytes */
-    private final int socketReceiveBuffer;
-
-    /* the client id used to identify this client in requests to the server */
-    private final String clientId;
-
-    /* a random offset to use when choosing nodes to avoid having all nodes choose the same node */
-    private final int nodeIndexOffset;
-
-    /* the current correlation id to use when sending requests to servers */
-    private int correlation;
-
-    /* true iff there is a metadata request that has been sent and for which we have not yet received a response */
-    private boolean metadataFetchInProgress;
-
-    /* the last timestamp when no broker node is available to connect */
-    private long lastNoNodeAvailableMs;
-
-    public NetworkClient(Selectable selector,
-                         Metadata metadata,
-                         String clientId,
-                         int maxInFlightRequestsPerConnection,
-                         long reconnectBackoffMs,
-                         int socketSendBuffer,
-                         int socketReceiveBuffer) {
-        this.selector = selector;
-        this.metadata = metadata;
-        this.clientId = clientId;
-        this.inFlightRequests = new InFlightRequests(maxInFlightRequestsPerConnection);
-        this.connectionStates = new ClusterConnectionStates(reconnectBackoffMs);
-        this.socketSendBuffer = socketSendBuffer;
-        this.socketReceiveBuffer = socketReceiveBuffer;
-        this.correlation = 0;
-        this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE);
-        this.metadataFetchInProgress = false;
-        this.lastNoNodeAvailableMs = 0;
-    }
-
-    /**
-     * Begin connecting to the given node, return true if we are already connected and ready to send to that node.
-     * 
-     * @param node The node to check
-     * @param now The current timestamp
-     * @return True if we are ready to send to the given node
-     */
-    @Override
-    public boolean ready(Node node, long now) {
-        if (isReady(node, now))
-            return true;
-
-        if (connectionStates.canConnect(node.idString(), now))
-            // if we are interested in sending to a node and we don't have a connection to it, initiate one
-            initiateConnect(node, now);
-
-        return false;
-    }
-
-    /**
-     * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
-     * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
-     * connections.
-     * 
-     * @param node The node to check
-     * @param now The current timestamp
-     * @return The number of milliseconds to wait.
-     */
-    @Override
-    public long connectionDelay(Node node, long now) {
-        return connectionStates.connectionDelay(node.idString(), now);
-    }
-
-    /**
-     * Check if the connection of the node has failed, based on the connection state. Such connection failure are
-     * usually transient and can be resumed in the next {@link #ready(Node, long)} }
-     * call, but there are cases where transient failures needs to be caught and re-acted upon.
-     *
-     * @param node the node to check
-     * @return true iff the connection has failed and the node is disconnected
-     */
-    @Override
-    public boolean connectionFailed(Node node) {
-        return connectionStates.connectionState(node.idString()).equals(ConnectionState.DISCONNECTED);
-    }
-
-    /**
-     * Check if the node with the given id is ready to send more requests.
-     * 
-     * @param node The node
-     * @param now The current time in ms
-     * @return true if the node is ready
-     */
-    @Override
-    public boolean isReady(Node node, long now) {
-        String nodeId = node.idString();
-        if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0)
-            // if we need to update our metadata now declare all requests unready to make metadata requests first
-            // priority
-            return false;
-        else
-            // otherwise we are ready if we are connected and can send more requests
-            return isSendable(nodeId);
-    }
-
-    /**
-     * Are we connected and ready and able to send more requests to the given connection?
-     * 
-     * @param node The node
-     */
-    private boolean isSendable(String node) {
-        return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node);
-    }
-
-    /**
-     * Return the state of the connection to the given node
-     * 
-     * @param node The node to check
-     * @return The connection state
-     */
-    public ConnectionState connectionState(String node) {
-        return connectionStates.connectionState(node);
-    }
-
-    /**
-     * Queue up the given request for sending. Requests can only be sent out to ready nodes.
-     * 
-     * @param request The request
-     */
-    @Override
-    public void send(ClientRequest request) {
-        String nodeId = request.request().destination();
-        if (!isSendable(nodeId))
-            throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready.");
-
-        this.inFlightRequests.add(request);
-        selector.send(request.request());
-    }
-
-    /**
-     * Do actual reads and writes to sockets.
-     * 
-     * @param timeout The maximum amount of time to wait (in ms) for responses if there are none immediately
-     * @param now The current time in milliseconds
-     * @return The list of responses received
-     */
-    @Override
-    public List<ClientResponse> poll(long timeout, long now) {
-        // should we update our metadata?
-        long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now);
-        long timeToNextReconnectAttempt = Math.max(this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now, 0);
-        long waitForMetadataFetch = this.metadataFetchInProgress ? Integer.MAX_VALUE : 0;
-        // if there is no node available to connect, back off refreshing metadata
-        long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt),
-                waitForMetadataFetch);
-        if (metadataTimeout == 0)
-            maybeUpdateMetadata(now);
-        // do the I/O
-        try {
-            this.selector.poll(Math.min(timeout, metadataTimeout));
-        } catch (IOException e) {
-            log.error("Unexpected error during I/O in producer network thread", e);
-        }
-
-        // process completed actions
-        List<ClientResponse> responses = new ArrayList<ClientResponse>();
-        handleCompletedSends(responses, now);
-        handleCompletedReceives(responses, now);
-        handleDisconnections(responses, now);
-        handleConnections();
-
-        // invoke callbacks
-        for (ClientResponse response : responses) {
-            if (response.request().hasCallback()) {
-                try {
-                    response.request().callback().onComplete(response);
-                } catch (Exception e) {
-                    log.error("Uncaught error in request completion:", e);
-                }
-            }
-        }
-
-        return responses;
-    }
-
-    /**
-     * Await all the outstanding responses for requests on the given connection
-     * 
-     * @param node The node to block on
-     * @param now The current time in ms
-     * @return All the collected responses
-     */
-    @Override
-    public List<ClientResponse> completeAll(String node, long now) {
-        try {
-            this.selector.muteAll();
-            this.selector.unmute(node);
-            List<ClientResponse> responses = new ArrayList<ClientResponse>();
-            while (inFlightRequestCount(node) > 0)
-                responses.addAll(poll(Integer.MAX_VALUE, now));
-            return responses;
-        } finally {
-            this.selector.unmuteAll();
-        }
-    }
-
-    /**
-     * Wait for all outstanding requests to complete.
-     */
-    @Override
-    public List<ClientResponse> completeAll(long now) {
-        List<ClientResponse> responses = new ArrayList<ClientResponse>();
-        while (inFlightRequestCount() > 0)
-            responses.addAll(poll(Integer.MAX_VALUE, now));
-        return responses;
-    }
-
-    /**
-     * Get the number of in-flight requests
-     */
-    @Override
-    public int inFlightRequestCount() {
-        return this.inFlightRequests.inFlightRequestCount();
-    }
-
-    /**
-     * Get the number of in-flight requests for a given node
-     */
-    @Override
-    public int inFlightRequestCount(String node) {
-        return this.inFlightRequests.inFlightRequestCount(node);
-    }
-
-    /**
-     * Generate a request header for the given API key
-     * 
-     * @param key The api key
-     * @return A request header with the appropriate client id and correlation id
-     */
-    @Override
-    public RequestHeader nextRequestHeader(ApiKeys key) {
-        return new RequestHeader(key.id, clientId, correlation++);
-    }
-
-    /**
-     * Interrupt the client if it is blocked waiting on I/O.
-     */
-    @Override
-    public void wakeup() {
-        this.selector.wakeup();
-    }
-
-    /**
-     * Close the network client
-     */
-    @Override
-    public void close() {
-        this.selector.close();
-    }
-
-    /**
-     * Choose the node with the fewest outstanding requests which is at least eligible for connection. This method will
-     * prefer a node with an existing connection, but will potentially choose a node for which we don't yet have a
-     * connection if all existing connections are in use. This method will never choose a node for which there is no
-     * existing connection and from which we have disconnected within the reconnect backoff period.
-     * 
-     * @return The node with the fewest in-flight requests.
-     */
-    public Node leastLoadedNode(long now) {
-        List<Node> nodes = this.metadata.fetch().nodes();
-        int inflight = Integer.MAX_VALUE;
-        Node found = null;
-        for (int i = 0; i < nodes.size(); i++) {
-            int idx = Utils.abs((this.nodeIndexOffset + i) % nodes.size());
-            Node node = nodes.get(idx);
-            int currInflight = this.inFlightRequests.inFlightRequestCount(node.idString());
-            if (currInflight == 0 && this.connectionStates.isConnected(node.idString())) {
-                // if we find an established connection with no in-flight requests we can stop right away
-                return node;
-            } else if (!this.connectionStates.isBlackedOut(node.idString(), now) && currInflight < inflight) {
-                // otherwise if this is the best we have found so far, record that
-                inflight = currInflight;
-                found = node;
-            }
-        }
-        return found;
-    }
-
-    /**
-     * Handle any completed request send. In particular if no response is expected consider the request complete.
-     * 
-     * @param responses The list of responses to update
-     * @param now The current time
-     */
-    private void handleCompletedSends(List<ClientResponse> responses, long now) {
-        // if no response is expected then when the send is completed, return it
-        for (Send send : this.selector.completedSends()) {
-            ClientRequest request = this.inFlightRequests.lastSent(send.destination());
-            if (!request.expectResponse()) {
-                this.inFlightRequests.completeLastSent(send.destination());
-                responses.add(new ClientResponse(request, now, false, null));
-            }
-        }
-    }
-
-    /**
-     * Handle any completed receives and update the response list with the responses received.
-     * 
-     * @param responses The list of responses to update
-     * @param now The current time
-     */
-    private void handleCompletedReceives(List<ClientResponse> responses, long now) {
-        for (NetworkReceive receive : this.selector.completedReceives()) {
-            String source = receive.source();
-            ClientRequest req = inFlightRequests.completeNext(source);
-            ResponseHeader header = ResponseHeader.parse(receive.payload());
-            short apiKey = req.request().header().apiKey();
-            Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload());
-            correlate(req.request().header(), header);
-            if (apiKey == ApiKeys.METADATA.id) {
-                handleMetadataResponse(req.request().header(), body, now);
-            } else {
-                // need to add body/header to response here
-                responses.add(new ClientResponse(req, now, false, body));
-            }
-        }
-    }
-
-    private void handleMetadataResponse(RequestHeader header, Struct body, long now) {
-        this.metadataFetchInProgress = false;
-        MetadataResponse response = new MetadataResponse(body);
-        Cluster cluster = response.cluster();
-        // check if any topics metadata failed to get updated
-        if (response.errors().size() > 0) {
-            log.warn("Error while fetching metadata with correlation id {} : {}", header.correlationId(), response.errors());
-        }
-        // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being
-        // created which means we will get errors and no nodes until it exists
-        if (cluster.nodes().size() > 0) {
-            this.metadata.update(cluster, now);
-        } else {
-            log.trace("Ignoring empty metadata response with correlation id {}.", header.correlationId());
-            this.metadata.failedUpdate(now);
-        }
-    }
-
-    /**
-     * Handle any disconnected connections
-     * 
-     * @param responses The list of responses that completed with the disconnection
-     * @param now The current time
-     */
-    private void handleDisconnections(List<ClientResponse> responses, long now) {
-        for (String node : this.selector.disconnected()) {
-            connectionStates.disconnected(node);
-            log.debug("Node {} disconnected.", node);
-            for (ClientRequest request : this.inFlightRequests.clearAll(node)) {
-                log.trace("Cancelled request {} due to node {} being disconnected", request, node);
-                ApiKeys requestKey = ApiKeys.forId(request.request().header().apiKey());
-                if (requestKey == ApiKeys.METADATA)
-                    metadataFetchInProgress = false;
-                else
-                    responses.add(new ClientResponse(request, now, true, null));
-            }
-        }
-        // we got a disconnect so we should probably refresh our metadata and see if that broker is dead
-        if (this.selector.disconnected().size() > 0)
-            this.metadata.requestUpdate();
-    }
-
-    /**
-     * Record any newly completed connections
-     */
-    private void handleConnections() {
-        for (String node : this.selector.connected()) {
-            log.debug("Completed connection to node {}", node);
-            this.connectionStates.connected(node);
-        }
-    }
-
-    /**
-     * Validate that the response corresponds to the request we expect or else explode
-     */
-    private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) {
-        if (requestHeader.correlationId() != responseHeader.correlationId())
-            throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId()
-                    + ") does not match request (" + requestHeader.correlationId() + ")");
-    }
-
-    /**
-     * Create a metadata request for the given topics
-     */
-    private ClientRequest metadataRequest(long now, String node, Set<String> topics) {
-        MetadataRequest metadata = new MetadataRequest(new ArrayList<String>(topics));
-        RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct());
-        return new ClientRequest(now, true, send, null);
-    }
-
-    /**
-     * Add a metadata request to the list of sends if we can make one
-     */
-    private void maybeUpdateMetadata(long now) {
-        // Beware that the behavior of this method and the computation of timeouts for poll() are
-        // highly dependent on the behavior of leastLoadedNode.
-        Node node = this.leastLoadedNode(now);
-        if (node == null) {
-            log.debug("Give up sending metadata request since no node is available");
-            // mark the timestamp for no node available to connect
-            this.lastNoNodeAvailableMs = now;
-            return;
-        }
-        String nodeConnectionId = node.idString();
-
-
-        if (connectionStates.isConnected(nodeConnectionId) && inFlightRequests.canSendMore(nodeConnectionId)) {
-            Set<String> topics = metadata.topics();
-            this.metadataFetchInProgress = true;
-            ClientRequest metadataRequest = metadataRequest(now, nodeConnectionId, topics);
-            log.debug("Sending metadata request {} to node {}", metadataRequest, node.id());
-            this.selector.send(metadataRequest.request());
-            this.inFlightRequests.add(metadataRequest);
-        } else if (connectionStates.canConnect(nodeConnectionId, now)) {
-            // we don't have a connection to this node right now, make one
-            log.debug("Initialize connection to node {} for sending metadata request", node.id());
-            initiateConnect(node, now);
-            // If initiateConnect failed immediately, this node will be put into blackout and we
-            // should allow immediately retrying in case there is another candidate node. If it
-            // is still connecting, the worst case is that we end up setting a longer timeout
-            // on the next round and then wait for the response.
-        } else { // connected, but can't send more OR connecting
-            // In either case, we just need to wait for a network event to let us know the selected
-            // connection might be usable again.
-            this.lastNoNodeAvailableMs = now;
-        }
-    }
-
-    /**
-     * Initiate a connection to the given node
-     */
-    private void initiateConnect(Node node, long now) {
-        String nodeConnectionId = node.idString();
-        try {
-            log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port());
-            this.connectionStates.connecting(nodeConnectionId, now);
-            selector.connect(nodeConnectionId,
-                             new InetSocketAddress(node.host(), node.port()),
-                             this.socketSendBuffer,
-                             this.socketReceiveBuffer);
-        } catch (IOException e) {
-            /* attempt failed, we'll try again after the backoff */
-            connectionStates.disconnected(nodeConnectionId);
-            /* maybe the problem is our metadata, update it */
-            metadata.requestUpdate();
-            log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/RequestCompletionHandler.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/RequestCompletionHandler.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/RequestCompletionHandler.java
deleted file mode 100644
index a275e96..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/RequestCompletionHandler.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.kafka_backport.clients;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A callback interface for attaching an action to be executed when a request is complete and the corresponding response
- * has been received. This handler will also be invoked if there is a disconnection while handling the request.
- */
-public interface RequestCompletionHandler {
-
-    public void onComplete(ClientResponse response);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/CommitType.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/CommitType.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/CommitType.java
deleted file mode 100644
index d261bd6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/CommitType.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public enum CommitType {
-    SYNC, ASYNC
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/Consumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/Consumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/Consumer.java
deleted file mode 100644
index 12b48ec..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/Consumer.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.kafka_backport.clients.consumer;
-
-import org.apache.flink.kafka_backport.common.Metric;
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.apache.flink.kafka_backport.common.PartitionInfo;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-
-import java.io.Closeable;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * @see KafkaConsumer
- * @see MockConsumer
- */
-public interface Consumer<K, V> extends Closeable {
-    
-    /**
-     * @see KafkaConsumer#subscriptions()
-     */
-    public Set<TopicPartition> subscriptions();
-
-    /**
-     * @see KafkaConsumer#subscribe(String...)
-     */
-    public void subscribe(String... topics);
-
-    /**
-     * @see KafkaConsumer#subscribe(TopicPartition...)
-     */
-    public void subscribe(TopicPartition... partitions);
-
-    /**
-     * @see KafkaConsumer#unsubscribe(String...)
-     */
-    public void unsubscribe(String... topics);
-
-    /**
-     * @see KafkaConsumer#unsubscribe(TopicPartition...)
-     */
-    public void unsubscribe(TopicPartition... partitions);
-
-    /**
-     * @see KafkaConsumer#poll(long)
-     */
-    public ConsumerRecords<K, V> poll(long timeout);
-
-    /**
-     * @see KafkaConsumer#commit(CommitType)
-     */
-    public void commit(CommitType commitType);
-
-    /**
-     * @see KafkaConsumer#commit(CommitType, ConsumerCommitCallback)
-     */
-    public void commit(CommitType commitType, ConsumerCommitCallback callback);
-
-    /**
-     * @see KafkaConsumer#commit(Map, CommitType)
-     */
-    public void commit(Map<TopicPartition, Long> offsets, CommitType commitType);
-
-    /**
-     * @see KafkaConsumer#commit(Map, CommitType, ConsumerCommitCallback)
-     */
-    public void commit(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback);
-
-    /**
-     * @see KafkaConsumer#seek(TopicPartition, long)
-     */
-    public void seek(TopicPartition partition, long offset);
-
-    /**
-     * @see KafkaConsumer#seekToBeginning(TopicPartition...)
-     */
-    public void seekToBeginning(TopicPartition... partitions);
-
-    /**
-     * @see KafkaConsumer#seekToEnd(TopicPartition...)
-     */
-    public void seekToEnd(TopicPartition... partitions);
-
-    /**
-     * @see KafkaConsumer#position(TopicPartition)
-     */
-    public long position(TopicPartition partition);
-
-    /**
-     * @see KafkaConsumer#committed(TopicPartition)
-     */
-    public long committed(TopicPartition partition);
-
-    /**
-     * @see KafkaConsumer#metrics()
-     */
-    public Map<MetricName, ? extends Metric> metrics();
-
-    /**
-     * @see KafkaConsumer#partitionsFor(String)
-     */
-    public List<PartitionInfo> partitionsFor(String topic);
-
-    /**
-     * @see KafkaConsumer#close()
-     */
-    public void close();
-
-    /**
-     * @see KafkaConsumer#wakeup()
-     */
-    public void wakeup();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerCommitCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerCommitCallback.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerCommitCallback.java
deleted file mode 100644
index 4d90bfc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerCommitCallback.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer;
-
-import org.apache.flink.kafka_backport.common.TopicPartition;
-
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A callback interface that the user can implement to trigger custom actions when a commit request completes. The callback
- * may be executed in any thread calling {@link Consumer#poll(long) poll()}.
- */
-public interface ConsumerCommitCallback {
-
-    /**
-     * A callback method the user can implement to provide asynchronous handling of commit request completion.
-     * This method will be called when the commit request sent to the server has been acknowledged.
-     *
-     * @param offsets A map of the offsets that this callback applies to
-     * @param exception The exception thrown during processing of the request, or null if the commit completed successfully
-     */
-    void onComplete(Map<TopicPartition, Long> offsets, Exception exception);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerConfig.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerConfig.java
deleted file mode 100644
index c1f79bf..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerConfig.java
+++ /dev/null
@@ -1,334 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer;
-
-import org.apache.flink.kafka_backport.clients.CommonClientConfigs;
-import org.apache.flink.kafka_backport.clients.consumer.internals.NoOpConsumerRebalanceCallback;
-import org.apache.flink.kafka_backport.common.config.AbstractConfig;
-import org.apache.flink.kafka_backport.common.config.ConfigDef;
-import org.apache.flink.kafka_backport.common.config.ConfigDef.Importance;
-import org.apache.flink.kafka_backport.common.config.ConfigDef.Type;
-import org.apache.flink.kafka_backport.common.serialization.Deserializer;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.flink.kafka_backport.common.config.ConfigDef.Range.atLeast;
-import static org.apache.flink.kafka_backport.common.config.ConfigDef.ValidString.in;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The consumer configuration keys
- */
-public class ConsumerConfig extends AbstractConfig {
-    private static final ConfigDef CONFIG;
-
-    /*
-     * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS
-     * THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
-     */
-
-    /**
-     * <code>group.id</code>
-     */
-    public static final String GROUP_ID_CONFIG = "group.id";
-    private static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using <code>subscribe(topic)</code> or the Kafka-based offset management strategy.";
-
-    /**
-     * <code>session.timeout.ms</code>
-     */
-    public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms";
-    private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities.";
-
-    /**
-     * <code>bootstrap.servers</code>
-     */
-    public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
-
-    /**
-     * <code>enable.auto.commit</code>
-     */
-    public static final String ENABLE_AUTO_COMMIT_CONFIG = "enable.auto.commit";
-    private static final String ENABLE_AUTO_COMMIT_DOC = "If true the consumer's offset will be periodically committed in the background.";
-
-    /**
-     * <code>auto.commit.interval.ms</code>
-     */
-    public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms";
-    private static final String AUTO_COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if <code>enable.auto.commit</code> is set to <code>true</code>.";
-
-    /**
-     * <code>partition.assignment.strategy</code>
-     */
-    public static final String PARTITION_ASSIGNMENT_STRATEGY_CONFIG = "partition.assignment.strategy";
-    private static final String PARTITION_ASSIGNMENT_STRATEGY_DOC = "The friendly name of the partition assignment strategy that the server will use to distribute partition ownership amongst consumer instances when group management is used";
-
-    /**
-     * <code>auto.offset.reset</code>
-     */
-    public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset";
-    private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted): <ul><li>smallest: automatically reset the offset to the smallest offset<li>largest: automatically reset the offset to the largest offset</li><li>none: throw exception to the consumer if no previous offset is found for the consumer's group</li><li>anything else: throw exception to the consumer.</li></ul>";
-
-    /**
-     * <code>fetch.min.bytes</code>
-     */
-    public static final String FETCH_MIN_BYTES_CONFIG = "fetch.min.bytes";
-    private static final String FETCH_MIN_BYTES_DOC = "The minimum amount of data the server should return for a fetch request. If insufficient data is available the request will wait for that much data to accumulate before answering the request. The default setting of 1 byte means that fetch requests are answered as soon as a single byte of data is available or the fetch request times out waiting for data to arrive. Setting this to something greater than 1 will cause the server to wait for larger amounts of data to accumulate which can improve server throughput a bit at the cost of some additional latency.";
-
-    /**
-     * <code>fetch.max.wait.ms</code>
-     */
-    public static final String FETCH_MAX_WAIT_MS_CONFIG = "fetch.max.wait.ms";
-    private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch.min.bytes.";
-
-    /** <code>metadata.max.age.ms</code> */
-    public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG;
-
-    /**
-     * <code>max.partition.fetch.bytes</code>
-     */
-    public static final String MAX_PARTITION_FETCH_BYTES_CONFIG = "max.partition.fetch.bytes";
-    private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server will return. The maximum total memory used for a request will be <code>#partitions * max.partition.fetch.bytes</code>. This size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large message on a certain partition.";
-
-    /** <code>send.buffer.bytes</code> */
-    public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG;
-
-    /** <code>receive.buffer.bytes</code> */
-    public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG;
-
-    /**
-     * <code>client.id</code>
-     */
-    public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
-
-    /**
-     * <code>reconnect.backoff.ms</code>
-     */
-    public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG;
-
-    /**
-     * <code>retry.backoff.ms</code>
-     */
-    public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG;
-
-    /**
-     * <code>metrics.sample.window.ms</code>
-     */
-    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
-
-    /**
-     * <code>metrics.num.samples</code>
-     */
-    public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG;
-
-    /**
-     * <code>metric.reporters</code>
-     */
-    public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG;
-
-    /**
-     * <code>rebalance.callback.class</code>
-     */
-    public static final String CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG = "rebalance.callback.class";
-    private static final String CONSUMER_REBALANCE_CALLBACK_CLASS_DOC = "A user-provided callback to execute when partition assignments change.";
-
-    /**
-     * <code>check.crcs</code>
-     */
-    public static final String CHECK_CRCS_CONFIG = "check.crcs";
-    private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance.";
-    
-    /** <code>key.deserializer</code> */
-    public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer";
-    private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the <code>Deserializer</code> interface.";
-
-    /** <code>value.deserializer</code> */
-    public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer";
-    private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the <code>Deserializer</code> interface.";
-
-    /** <code>connections.max.idle.ms</code> */
-    public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;
-
-
-    static {
-        CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
-                                        Type.LIST,
-                                        Importance.HIGH,
-                                        CommonClientConfigs.BOOSTRAP_SERVERS_DOC)
-                                .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC)
-                                .define(SESSION_TIMEOUT_MS_CONFIG,
-                                        Type.INT,
-                                        30000,
-                                        Importance.HIGH,
-                                        SESSION_TIMEOUT_MS_DOC)
-                                .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
-                                        Type.STRING,
-                                        "range",
-                                        in("range", "roundrobin"),
-                                        Importance.MEDIUM,
-                                        PARTITION_ASSIGNMENT_STRATEGY_DOC)
-                                .define(METADATA_MAX_AGE_CONFIG,
-                                        Type.LONG,
-                                        5 * 60 * 1000,
-                                        atLeast(0),
-                                        Importance.LOW,
-                                        CommonClientConfigs.METADATA_MAX_AGE_DOC)
-                                .define(ENABLE_AUTO_COMMIT_CONFIG,
-                                        Type.BOOLEAN,
-                                        true,
-                                        Importance.MEDIUM,
-                                        ENABLE_AUTO_COMMIT_DOC)
-                                .define(AUTO_COMMIT_INTERVAL_MS_CONFIG,
-                                        Type.LONG,
-                                        5000,
-                                        atLeast(0),
-                                        Importance.LOW,
-                                        AUTO_COMMIT_INTERVAL_MS_DOC)
-                                .define(CLIENT_ID_CONFIG,
-                                        Type.STRING,
-                                        "",
-                                        Importance.LOW,
-                                        CommonClientConfigs.CLIENT_ID_DOC)
-                                .define(MAX_PARTITION_FETCH_BYTES_CONFIG,
-                                        Type.INT,
-                                        1 * 1024 * 1024,
-                                        atLeast(0),
-                                        Importance.HIGH,
-                                        MAX_PARTITION_FETCH_BYTES_DOC)
-                                .define(SEND_BUFFER_CONFIG,
-                                        Type.INT,
-                                        128 * 1024,
-                                        atLeast(0),
-                                        Importance.MEDIUM,
-                                        CommonClientConfigs.SEND_BUFFER_DOC)
-                                .define(RECEIVE_BUFFER_CONFIG,
-                                        Type.INT,
-                                        32 * 1024,
-                                        atLeast(0),
-                                        Importance.MEDIUM,
-                                        CommonClientConfigs.RECEIVE_BUFFER_DOC)
-                                .define(FETCH_MIN_BYTES_CONFIG,
-                                        Type.INT,
-                                        1024,
-                                        atLeast(0),
-                                        Importance.HIGH,
-                                        FETCH_MIN_BYTES_DOC)
-                                .define(FETCH_MAX_WAIT_MS_CONFIG,
-                                        Type.INT,
-                                        500,
-                                        atLeast(0),
-                                        Importance.LOW,
-                                        FETCH_MAX_WAIT_MS_DOC)
-                                .define(RECONNECT_BACKOFF_MS_CONFIG,
-                                        Type.LONG,
-                                        50L,
-                                        atLeast(0L),
-                                        Importance.LOW,
-                                        CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC)
-                                .define(RETRY_BACKOFF_MS_CONFIG,
-                                        Type.LONG,
-                                        100L,
-                                        atLeast(0L),
-                                        Importance.LOW,
-                                        CommonClientConfigs.RETRY_BACKOFF_MS_DOC)
-                                .define(AUTO_OFFSET_RESET_CONFIG,
-                                        Type.STRING,
-                                        "latest",
-                                        in("latest", "earliest", "none"),
-                                        Importance.MEDIUM,
-                                        AUTO_OFFSET_RESET_DOC)
-                                .define(CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
-                                        Type.CLASS,
-                                        NoOpConsumerRebalanceCallback.class,
-                                        Importance.LOW,
-                                        CONSUMER_REBALANCE_CALLBACK_CLASS_DOC)
-                                .define(CHECK_CRCS_CONFIG,
-                                        Type.BOOLEAN,
-                                        true,
-                                        Importance.LOW,
-                                        CHECK_CRCS_DOC)                                
-                                .define(METRICS_SAMPLE_WINDOW_MS_CONFIG,
-                                        Type.LONG,
-                                        30000,
-                                        atLeast(0),
-                                        Importance.LOW,
-                                        CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC)
-                                .define(METRICS_NUM_SAMPLES_CONFIG,
-                                        Type.INT,
-                                        2,
-                                        atLeast(1),
-                                        Importance.LOW,
-                                        CommonClientConfigs.METRICS_NUM_SAMPLES_DOC)
-                                .define(METRIC_REPORTER_CLASSES_CONFIG,
-                                        Type.LIST,
-                                        "",
-                                        Importance.LOW,
-                                        CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC)
-                                .define(KEY_DESERIALIZER_CLASS_CONFIG,
-                                        Type.CLASS,
-                                        Importance.HIGH,
-                                        KEY_DESERIALIZER_CLASS_DOC)
-                                .define(VALUE_DESERIALIZER_CLASS_CONFIG,
-                                        Type.CLASS,
-                                        Importance.HIGH,
-                                        VALUE_DESERIALIZER_CLASS_DOC)
-                                /* default is set to be a bit lower than the server default (10 min), to avoid both client and server closing connection at same time */
-                                .define(CONNECTIONS_MAX_IDLE_MS_CONFIG,
-                                        Type.LONG,
-                                        9 * 60 * 1000,
-                                        Importance.MEDIUM,
-                                        CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC);
-    }
-
-    public static Map<String, Object> addDeserializerToConfig(Map<String, Object> configs,
-                                                              Deserializer<?> keyDeserializer,
-                                                              Deserializer<?> valueDeserializer) {
-        Map<String, Object> newConfigs = new HashMap<String, Object>();
-        newConfigs.putAll(configs);
-        if (keyDeserializer != null)
-            newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass());
-        if (keyDeserializer != null)
-            newConfigs.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass());
-        return newConfigs;
-    }
-
-    public static Properties addDeserializerToConfig(Properties properties,
-                                                     Deserializer<?> keyDeserializer,
-                                                     Deserializer<?> valueDeserializer) {
-        Properties newProperties = new Properties();
-        newProperties.putAll(properties);
-        if (keyDeserializer != null)
-            newProperties.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName());
-        if (keyDeserializer != null)
-            newProperties.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName());
-        return newProperties;
-    }
-
-    public ConsumerConfig(Map<?, ?> props) {
-        super(CONFIG, props);
-    }
-
-    public static void main(String[] args) {
-        System.out.println(CONFIG.toHtmlTable());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRebalanceCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRebalanceCallback.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRebalanceCallback.java
deleted file mode 100644
index 8f2cd75..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRebalanceCallback.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.kafka_backport.clients.consumer;
-
-import org.apache.flink.kafka_backport.common.TopicPartition;
-
-import java.util.Collection;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A callback interface that the user can implement to trigger custom actions when the set of partitions assigned to the
- * consumer changes.
- * <p>
- * This is applicable when the consumer is having Kafka auto-manage group membership, if the consumer's directly subscribe to partitions
- * those partitions will never be reassigned and this callback is not applicable.
- * <p>
- * When Kafka is managing the group membership, a partition re-assignment will be triggered any time the members of the group changes or the subscription
- * of the members changes. This can occur when processes die, new process instances are added or old instances come back to life after failure.
- * <p>
- * There are many uses for this functionality. One common use is saving offsets in a custom store. By saving offsets in
- * the {@link #onPartitionsRevoked(Consumer, Collection)} call we can ensure that any time partition assignment changes
- * the offset gets saved.
- * <p>
- * Another use is flushing out any kind of cache of intermediate results the consumer may be keeping. For example,
- * consider a case where the consumer is subscribed to a topic containing user page views, and the goal is to count the
- * number of page views per users for each five minute window. Let's say the topic is partitioned by the user id so that
- * all events for a particular user will go to a single consumer instance. The consumer can keep in memory a running
- * tally of actions per user and only flush these out to a remote data store when it's cache gets to big. However if a
- * partition is reassigned it may want to automatically trigger a flush of this cache, before the new owner takes over
- * consumption.
- * <p>
- * This callback will execute in the user thread as part of the {@link Consumer#poll(long) poll(long)} call whenever partition assignment changes.
- * <p>
- * It is guaranteed that all consumer processes will invoke {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} prior to 
- * any process invoking {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned}. So if offsets or other state is saved in the 
- * {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} call it is guaranteed to be saved by the time the process taking over that
- * partition has their {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned} callback called to load the state.
- * <p>
- * Here is pseudo-code for a callback implementation for saving offsets:
- * <pre>
- * {@code
- *   public class SaveOffsetsOnRebalance implements ConsumerRebalanceCallback {
- *       public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
- *           // read the offsets from an external store using some custom code not described here
- *           for(TopicPartition partition: partitions)
- *              consumer.position(partition, readOffsetFromExternalStore(partition));
- *       }      
- *       public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
- *           // save the offsets in an external store using some custom code not described here
- *           for(TopicPartition partition: partitions)
- *              saveOffsetInExternalStore(consumer.position(partition));
- *       }
- *   }
- * }
- * </pre>
- */
-public interface ConsumerRebalanceCallback {
-
-    /**
-     * A callback method the user can implement to provide handling of customized offsets on completion of a successful
-     * partition re-assignement. This method will be called after an offset re-assignement completes and before the
-     * consumer starts fetching data.
-     * <p>
-     * It is guaranteed that all the processes in a consumer group will execute their
-     * {@link #onPartitionsRevoked(Consumer, Collection)} callback before any instance executes its
-     * {@link #onPartitionsAssigned(Consumer, Collection)} callback.
-     *
-     * @param consumer Reference to the consumer for convenience
-     * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously
-     *            assigned to the consumer)
-     */
-    public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
-
-    /**
-     * A callback method the user can implement to provide handling of offset commits to a customized store on the start
-     * of a rebalance operation. This method will be called before a rebalance operation starts and after the consumer
-     * stops fetching data. It is recommended that offsets should be committed in this callback to either Kafka or a
-     * custom offset store to prevent duplicate data
-     * <p>
-     * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}
-     *
-     * @param consumer  Reference to the consumer for convenience
-     * @param partitions The list of partitions that were assigned to the consumer on the last rebalance
-     */
-    public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecord.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecord.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecord.java
deleted file mode 100644
index 44096ee..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecord.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the
- * record is being received and an offset that points to the record in a Kafka partition.
- */
-public final class ConsumerRecord<K, V> {
-    private final String topic;
-    private final int partition;
-    private final long offset;
-    private final K key;
-    private final V value;
-
-    /**
-     * Create a record with no key
-     * 
-     * @param topic The topic this record is received from
-     * @param partition The partition of the topic this record is received from
-     * @param offset The offset of this record in the corresponding Kafka partition
-     * @param value The record contents
-     */
-    public ConsumerRecord(String topic, int partition, long offset, K key, V value) {
-        if (topic == null)
-            throw new IllegalArgumentException("Topic cannot be null");
-        this.topic = topic;
-        this.partition = partition;
-        this.offset = offset;
-        this.key = key;
-        this.value = value;
-    }
-
-    /**
-     * The topic this record is received from
-     */
-    public String topic() {
-        return this.topic;
-    }
-
-    /**
-     * The partition from which this record is received
-     */
-    public int partition() {
-        return this.partition;
-    }
-
-    /**
-     * The key (or null if no key is specified)
-     */
-    public K key() {
-        return key;
-    }
-
-    /**
-     * The value
-     */
-    public V value() {
-        return value;
-    }
-
-    /**
-     * The position of this record in the corresponding Kafka partition.
-     */
-    public long offset() {
-        return offset;
-    }
-
-    @Override
-    public String toString() {
-        return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset()
-                + ", key = " + key + ", value = " + value + ")";
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecords.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecords.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecords.java
deleted file mode 100644
index a9cfc84..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerRecords.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.kafka_backport.clients.consumer;
-
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.utils.AbstractIterator;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A container that holds the list {@link ConsumerRecord} per partition for a
- * particular topic. There is one for every topic returned by a
- * {@link Consumer#poll(long)} operation.
- */
-public class ConsumerRecords<K, V> implements Iterable<ConsumerRecord<K, V>> {
-    public static final ConsumerRecords<Object, Object> EMPTY =
-            new ConsumerRecords<Object, Object>(Collections.EMPTY_MAP);
-
-    private final Map<TopicPartition, List<ConsumerRecord<K, V>>> records;
-
-    public ConsumerRecords(Map<TopicPartition, List<ConsumerRecord<K, V>>> records) {
-        this.records = records;
-    }
-
-    /**
-     * Get just the records for the given partition
-     * 
-     * @param partition The partition to get records for
-     */
-    public Iterable<ConsumerRecord<K, V>> records(TopicPartition partition) {
-        List<ConsumerRecord<K, V>> recs = this.records.get(partition);
-        if (recs == null)
-            return Collections.emptyList();
-        else
-            return recs;
-    }
-
-    /**
-     * Get just the records for the given topic
-     */
-    public Iterable<ConsumerRecord<K, V>> records(String topic) {
-        if (topic == null)
-            throw new IllegalArgumentException("Topic must be non-null.");
-        List<List<ConsumerRecord<K, V>>> recs = new ArrayList<List<ConsumerRecord<K, V>>>();
-        for (Map.Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : records.entrySet()) {
-            if (entry.getKey().topic().equals(topic))
-                recs.add(entry.getValue());
-        }
-        return new ConcatenatedIterable<K, V>(recs);
-    }
-
-    @Override
-    public Iterator<ConsumerRecord<K, V>> iterator() {
-        return new ConcatenatedIterable<K, V>(records.values()).iterator();
-    }
-    
-    /**
-     * The number of records for all topics
-     */
-    public int count() {
-        int count = 0;
-        for (List<ConsumerRecord<K, V>> recs: this.records.values())
-            count += recs.size();
-        return count;
-    }
-
-    private static class ConcatenatedIterable<K, V> implements Iterable<ConsumerRecord<K, V>> {
-
-        private final Iterable<? extends Iterable<ConsumerRecord<K, V>>> iterables;
-
-        public ConcatenatedIterable(Iterable<? extends Iterable<ConsumerRecord<K, V>>> iterables) {
-            this.iterables = iterables;
-        }
-
-        @Override
-        public Iterator<ConsumerRecord<K, V>> iterator() {
-            return new AbstractIterator<ConsumerRecord<K, V>>() {
-                Iterator<? extends Iterable<ConsumerRecord<K, V>>> iters = iterables.iterator();
-                Iterator<ConsumerRecord<K, V>> current;
-
-                public ConsumerRecord<K, V> makeNext() {
-                    if (current == null || !current.hasNext()) {
-                        if (iters.hasNext())
-                            current = iters.next().iterator();
-                        else
-                            return allDone();
-                    }
-                    return current.next();
-                }
-            };
-        }
-    }
-
-    public boolean isEmpty() {
-        return records.isEmpty();
-    }
-
-    @SuppressWarnings("unchecked")
-    public static <K, V> ConsumerRecords<K, V> empty() {
-        return (ConsumerRecords<K, V>) EMPTY;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerWakeupException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerWakeupException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerWakeupException.java
deleted file mode 100644
index 7185c87..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/ConsumerWakeupException.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.kafka_backport.clients.consumer;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ConsumerWakeupException extends KafkaException {
-    private static final long serialVersionUID = 1L;
-
-}


[19/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ConsumerCoordinatorNotAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ConsumerCoordinatorNotAvailableException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ConsumerCoordinatorNotAvailableException.java
new file mode 100644
index 0000000..1add30e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/ConsumerCoordinatorNotAvailableException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has
+ * not yet been created.
+ */
+public class ConsumerCoordinatorNotAvailableException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public ConsumerCoordinatorNotAvailableException() {
+        super();
+    }
+
+    public ConsumerCoordinatorNotAvailableException(String message) {
+        super(message);
+    }
+
+    public ConsumerCoordinatorNotAvailableException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ConsumerCoordinatorNotAvailableException(Throwable cause) {
+        super(cause);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/CorruptRecordException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/CorruptRecordException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/CorruptRecordException.java
new file mode 100644
index 0000000..d5771df
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/CorruptRecordException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This exception indicates a record has failed it's internal CRC check, this generally indicates network or disk
+ * corruption.
+ */
+public class CorruptRecordException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public CorruptRecordException() {
+        super("This message has failed it's CRC checksum or is otherwise corrupt.");
+    }
+
+    public CorruptRecordException(String message) {
+        super(message);
+    }
+
+    public CorruptRecordException(Throwable cause) {
+        super(cause);
+    }
+
+    public CorruptRecordException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/DisconnectException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/DisconnectException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/DisconnectException.java
new file mode 100644
index 0000000..1c048d3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/DisconnectException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Server disconnected before a request could be completed.
+ */
+public class DisconnectException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public DisconnectException() {
+        super();
+    }
+
+    public DisconnectException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public DisconnectException(String message) {
+        super(message);
+    }
+
+    public DisconnectException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/IllegalGenerationException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/IllegalGenerationException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/IllegalGenerationException.java
new file mode 100644
index 0000000..ae46b5f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/IllegalGenerationException.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class IllegalGenerationException extends RetriableException {
+    private static final long serialVersionUID = 1L;
+
+    public IllegalGenerationException() {
+        super();
+    }
+
+    public IllegalGenerationException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public IllegalGenerationException(String message) {
+        super(message);
+    }
+
+    public IllegalGenerationException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InterruptException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InterruptException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InterruptException.java
new file mode 100644
index 0000000..f7949f4
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InterruptException.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.kafka_backport.common.errors;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * An unchecked wrapper for InterruptedException
+ */
+public class InterruptException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+    
+    public InterruptException(InterruptedException cause) {
+        super(cause);
+        Thread.currentThread().interrupt();
+    }
+    
+    public InterruptException(String message, InterruptedException cause) {
+        super(message, cause);
+        Thread.currentThread().interrupt();
+    }
+
+    public InterruptException(String message) {
+        super(message, new InterruptedException());
+        Thread.currentThread().interrupt();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidMetadataException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidMetadataException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidMetadataException.java
new file mode 100644
index 0000000..710a391
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidMetadataException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * An exception that may indicate the client's metadata is out of date
+ */
+public abstract class InvalidMetadataException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public InvalidMetadataException() {
+        super();
+    }
+
+    public InvalidMetadataException(String message) {
+        super(message);
+    }
+
+    public InvalidMetadataException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public InvalidMetadataException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidRequiredAcksException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidRequiredAcksException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidRequiredAcksException.java
new file mode 100644
index 0000000..a3cd167
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidRequiredAcksException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class InvalidRequiredAcksException extends ApiException {
+    private static final long serialVersionUID = 1L;
+
+    public InvalidRequiredAcksException(String message) {
+        super(message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidTopicException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidTopicException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidTopicException.java
new file mode 100644
index 0000000..e0c5a41
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/InvalidTopicException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The client has attempted to perform an operation on an invalid topic.
+ */
+public class InvalidTopicException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public InvalidTopicException() {
+        super();
+    }
+
+    public InvalidTopicException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public InvalidTopicException(String message) {
+        super(message);
+    }
+
+    public InvalidTopicException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/LeaderNotAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/LeaderNotAvailableException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/LeaderNotAvailableException.java
new file mode 100644
index 0000000..701a3c6
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/LeaderNotAvailableException.java
@@ -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.
+ */
+package org.apache.flink.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * There is no currently available leader for the given partition (either because a leadership election is in progress
+ * or because all replicas are down).
+ */
+public class LeaderNotAvailableException extends InvalidMetadataException {
+
+    private static final long serialVersionUID = 1L;
+
+    public LeaderNotAvailableException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NetworkException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NetworkException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NetworkException.java
new file mode 100644
index 0000000..ceca78f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NetworkException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A misc. network-related IOException occurred when making a request. This could be because the client's metadata is
+ * out of date and it is making a request to a node that is now dead.
+ */
+public class NetworkException extends InvalidMetadataException {
+
+    private static final long serialVersionUID = 1L;
+
+    public NetworkException() {
+        super();
+    }
+
+    public NetworkException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public NetworkException(String message) {
+        super(message);
+    }
+
+    public NetworkException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotCoordinatorForConsumerException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotCoordinatorForConsumerException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotCoordinatorForConsumerException.java
new file mode 100644
index 0000000..3aea94b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotCoordinatorForConsumerException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is
+ * not a coordinator for.
+ */
+public class NotCoordinatorForConsumerException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public NotCoordinatorForConsumerException() {
+        super();
+    }
+
+    public NotCoordinatorForConsumerException(String message) {
+        super(message);
+    }
+
+    public NotCoordinatorForConsumerException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public NotCoordinatorForConsumerException(Throwable cause) {
+        super(cause);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasAfterAppendException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasAfterAppendException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasAfterAppendException.java
new file mode 100644
index 0000000..c2f8203
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasAfterAppendException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Number of insync replicas for the partition is lower than min.insync.replicas This exception is raised when the low
+ * ISR size is discovered *after* the message was already appended to the log. Producer retries will cause duplicates.
+ */
+public class NotEnoughReplicasAfterAppendException extends RetriableException {
+    private static final long serialVersionUID = 1L;
+
+    public NotEnoughReplicasAfterAppendException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasException.java
new file mode 100644
index 0000000..93eb850
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotEnoughReplicasException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Number of insync replicas for the partition is lower than min.insync.replicas
+ */
+public class NotEnoughReplicasException extends RetriableException {
+    private static final long serialVersionUID = 1L;
+
+    public NotEnoughReplicasException() {
+        super();
+    }
+
+    public NotEnoughReplicasException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public NotEnoughReplicasException(String message) {
+        super(message);
+    }
+
+    public NotEnoughReplicasException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotLeaderForPartitionException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotLeaderForPartitionException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotLeaderForPartitionException.java
new file mode 100644
index 0000000..a7dafb4
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/NotLeaderForPartitionException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This server is not the leader for the given partition
+ */
+public class NotLeaderForPartitionException extends InvalidMetadataException {
+
+    private static final long serialVersionUID = 1L;
+
+    public NotLeaderForPartitionException() {
+        super();
+    }
+
+    public NotLeaderForPartitionException(String message) {
+        super(message);
+    }
+
+    public NotLeaderForPartitionException(Throwable cause) {
+        super(cause);
+    }
+
+    public NotLeaderForPartitionException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetLoadInProgressException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetLoadInProgressException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetLoadInProgressException.java
new file mode 100644
index 0000000..f6a17b9
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetLoadInProgressException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change
+ * for that offsets topic partition).
+ */
+public class OffsetLoadInProgressException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public OffsetLoadInProgressException() {
+        super();
+    }
+
+    public OffsetLoadInProgressException(String message) {
+        super(message);
+    }
+
+    public OffsetLoadInProgressException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public OffsetLoadInProgressException(Throwable cause) {
+        super(cause);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetMetadataTooLarge.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetMetadataTooLarge.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetMetadataTooLarge.java
new file mode 100644
index 0000000..6a57b5d
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetMetadataTooLarge.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The client has tried to save its offset with associated metadata larger than the maximum size allowed by the server.
+ */
+public class OffsetMetadataTooLarge extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public OffsetMetadataTooLarge() {
+    }
+
+    public OffsetMetadataTooLarge(String message) {
+        super(message);
+    }
+
+    public OffsetMetadataTooLarge(Throwable cause) {
+        super(cause);
+    }
+
+    public OffsetMetadataTooLarge(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetOutOfRangeException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetOutOfRangeException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetOutOfRangeException.java
new file mode 100644
index 0000000..dfc35b7
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/OffsetOutOfRangeException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This offset is either larger or smaller than the range of offsets the server has for the given partition.
+ * 
+ */
+public class OffsetOutOfRangeException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public OffsetOutOfRangeException() {
+    }
+
+    public OffsetOutOfRangeException(String message) {
+        super(message);
+    }
+
+    public OffsetOutOfRangeException(Throwable cause) {
+        super(cause);
+    }
+
+    public OffsetOutOfRangeException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordBatchTooLargeException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordBatchTooLargeException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordBatchTooLargeException.java
new file mode 100644
index 0000000..360f042
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordBatchTooLargeException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This record batch is larger than the maximum allowable size
+ */
+public class RecordBatchTooLargeException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public RecordBatchTooLargeException() {
+        super();
+    }
+
+    public RecordBatchTooLargeException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public RecordBatchTooLargeException(String message) {
+        super(message);
+    }
+
+    public RecordBatchTooLargeException(Throwable cause) {
+        super(cause);
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordTooLargeException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordTooLargeException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordTooLargeException.java
new file mode 100644
index 0000000..0fd5a5f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RecordTooLargeException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This record is larger than the maximum allowable size
+ */
+public class RecordTooLargeException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public RecordTooLargeException() {
+        super();
+    }
+
+    public RecordTooLargeException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public RecordTooLargeException(String message) {
+        super(message);
+    }
+
+    public RecordTooLargeException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RetriableException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RetriableException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RetriableException.java
new file mode 100644
index 0000000..419174f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/RetriableException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A retryable exception is a transient exception that if retried may succeed.
+ */
+public abstract class RetriableException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public RetriableException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public RetriableException(String message) {
+        super(message);
+    }
+
+    public RetriableException(Throwable cause) {
+        super(cause);
+    }
+
+    public RetriableException() {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/SerializationException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/SerializationException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/SerializationException.java
new file mode 100644
index 0000000..40f07fc
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/SerializationException.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.kafka_backport.common.errors;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ *  Any exception during serialization in the producer
+ */
+public class SerializationException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public SerializationException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public SerializationException(String message) {
+        super(message);
+    }
+
+    public SerializationException(Throwable cause) {
+        super(cause);
+    }
+
+    public SerializationException() {
+        super();
+    }
+
+    /* avoid the expensive and useless stack trace for serialization exceptions */
+    @Override
+    public Throwable fillInStackTrace() {
+        return this;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/TimeoutException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/TimeoutException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/TimeoutException.java
new file mode 100644
index 0000000..4fd5a32
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/TimeoutException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Indicates that a request timed out.
+ */
+public class TimeoutException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public TimeoutException() {
+        super();
+    }
+
+    public TimeoutException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public TimeoutException(String message) {
+        super(message);
+    }
+
+    public TimeoutException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownConsumerIdException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownConsumerIdException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownConsumerIdException.java
new file mode 100644
index 0000000..a86997c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownConsumerIdException.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class UnknownConsumerIdException extends RetriableException {
+    private static final long serialVersionUID = 1L;
+
+    public UnknownConsumerIdException() {
+        super();
+    }
+
+    public UnknownConsumerIdException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public UnknownConsumerIdException(String message) {
+        super(message);
+    }
+
+    public UnknownConsumerIdException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownServerException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownServerException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownServerException.java
new file mode 100644
index 0000000..423e8d3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownServerException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * An error occurred on the server for which the client doesn't have a corresponding error code. This is generally an
+ * unexpected error.
+ * 
+ */
+public class UnknownServerException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public UnknownServerException() {
+    }
+
+    public UnknownServerException(String message) {
+        super(message);
+    }
+
+    public UnknownServerException(Throwable cause) {
+        super(cause);
+    }
+
+    public UnknownServerException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownTopicOrPartitionException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownTopicOrPartitionException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownTopicOrPartitionException.java
new file mode 100644
index 0000000..2b1a733
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/errors/UnknownTopicOrPartitionException.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.kafka_backport.common.errors;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This topic/partition doesn't exist
+ */
+public class UnknownTopicOrPartitionException extends InvalidMetadataException {
+
+    private static final long serialVersionUID = 1L;
+
+    public UnknownTopicOrPartitionException() {
+    }
+
+    public UnknownTopicOrPartitionException(String message) {
+        super(message);
+    }
+
+    public UnknownTopicOrPartitionException(Throwable throwable) {
+        super(throwable);
+    }
+
+    public UnknownTopicOrPartitionException(String message, Throwable throwable) {
+        super(message, throwable);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/CompoundStat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/CompoundStat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/CompoundStat.java
new file mode 100644
index 0000000..984e41e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/CompoundStat.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.metrics;
+
+import org.apache.flink.kafka_backport.common.MetricName;
+
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A compound stat is a stat where a single measurement and associated data structure feeds many metrics. This is the
+ * example for a histogram which has many associated percentiles.
+ */
+public interface CompoundStat extends Stat {
+
+    public List<NamedMeasurable> stats();
+
+    public static class NamedMeasurable {
+
+        private final MetricName name;
+        private final Measurable stat;
+
+        public NamedMeasurable(MetricName name, Measurable stat) {
+            super();
+            this.name = name;
+            this.stat = stat;
+        }
+
+        public MetricName name() {
+            return name;
+        }
+
+        public Measurable stat() {
+            return stat;
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/JmxReporter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/JmxReporter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/JmxReporter.java
new file mode 100644
index 0000000..5360efa
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/JmxReporter.java
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.metrics;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.management.Attribute;
+import javax.management.AttributeList;
+import javax.management.AttributeNotFoundException;
+import javax.management.DynamicMBean;
+import javax.management.InvalidAttributeValueException;
+import javax.management.JMException;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanException;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import java.lang.management.ManagementFactory;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Register metrics in JMX as dynamic mbeans based on the metric names
+ */
+public class JmxReporter implements MetricsReporter {
+
+    private static final Logger log = LoggerFactory.getLogger(JmxReporter.class);
+    private static final Object LOCK = new Object();
+    private String prefix;
+    private final Map<String, KafkaMbean> mbeans = new HashMap<String, KafkaMbean>();
+
+    public JmxReporter() {
+        this("");
+    }
+
+    /**
+     * Create a JMX reporter that prefixes all metrics with the given string.
+     */
+    public JmxReporter(String prefix) {
+        this.prefix = prefix;
+    }
+
+    @Override
+    public void configure(Map<String, ?> configs) {}
+
+    @Override
+    public void init(List<KafkaMetric> metrics) {
+        synchronized (LOCK) {
+            for (KafkaMetric metric : metrics)
+                addAttribute(metric);
+            for (KafkaMbean mbean : mbeans.values())
+                reregister(mbean);
+        }
+    }
+
+    @Override
+    public void metricChange(KafkaMetric metric) {
+        synchronized (LOCK) {
+            KafkaMbean mbean = addAttribute(metric);
+            reregister(mbean);
+        }
+    }
+
+    private KafkaMbean addAttribute(KafkaMetric metric) {
+        try {
+            MetricName metricName = metric.metricName();
+            String mBeanName = getMBeanName(metricName);
+            if (!this.mbeans.containsKey(mBeanName))
+                mbeans.put(mBeanName, new KafkaMbean(mBeanName));
+            KafkaMbean mbean = this.mbeans.get(mBeanName);
+            mbean.setAttribute(metricName.name(), metric);
+            return mbean;
+        } catch (JMException e) {
+            throw new KafkaException("Error creating mbean attribute for metricName :" + metric.metricName(), e);
+        }
+    }
+
+    /**
+     * @param metricName
+     * @return standard JMX MBean name in the following format domainName:type=metricType,key1=val1,key2=val2
+     */
+    private String getMBeanName(MetricName metricName) {
+        StringBuilder mBeanName = new StringBuilder();
+        mBeanName.append(prefix);
+        mBeanName.append(":type=");
+        mBeanName.append(metricName.group());
+        for (Map.Entry<String, String> entry : metricName.tags().entrySet()) {
+            if (entry.getKey().length() <= 0 || entry.getValue().length() <= 0)
+                continue;
+            mBeanName.append(",");
+            mBeanName.append(entry.getKey());
+            mBeanName.append("=");
+            mBeanName.append(entry.getValue());
+        }
+        return mBeanName.toString();
+    }
+
+    public void close() {
+        synchronized (LOCK) {
+            for (KafkaMbean mbean : this.mbeans.values())
+                unregister(mbean);
+        }
+    }
+
+    private void unregister(KafkaMbean mbean) {
+        MBeanServer server = ManagementFactory.getPlatformMBeanServer();
+        try {
+            if (server.isRegistered(mbean.name()))
+                server.unregisterMBean(mbean.name());
+        } catch (JMException e) {
+            throw new KafkaException("Error unregistering mbean", e);
+        }
+    }
+
+    private void reregister(KafkaMbean mbean) {
+        unregister(mbean);
+        try {
+            ManagementFactory.getPlatformMBeanServer().registerMBean(mbean, mbean.name());
+        } catch (JMException e) {
+            throw new KafkaException("Error registering mbean " + mbean.name(), e);
+        }
+    }
+
+    private static class KafkaMbean implements DynamicMBean {
+        private final ObjectName objectName;
+        private final Map<String, KafkaMetric> metrics;
+
+        public KafkaMbean(String mbeanName) throws MalformedObjectNameException {
+            this.metrics = new HashMap<String, KafkaMetric>();
+            this.objectName = new ObjectName(mbeanName);
+        }
+
+        public ObjectName name() {
+            return objectName;
+        }
+
+        public void setAttribute(String name, KafkaMetric metric) {
+            this.metrics.put(name, metric);
+        }
+
+        @Override
+        public Object getAttribute(String name) throws AttributeNotFoundException, MBeanException, ReflectionException {
+            if (this.metrics.containsKey(name))
+                return this.metrics.get(name).value();
+            else
+                throw new AttributeNotFoundException("Could not find attribute " + name);
+        }
+
+        @Override
+        public AttributeList getAttributes(String[] names) {
+            try {
+                AttributeList list = new AttributeList();
+                for (String name : names)
+                    list.add(new Attribute(name, getAttribute(name)));
+                return list;
+            } catch (Exception e) {
+                log.error("Error getting JMX attribute: ", e);
+                return new AttributeList();
+            }
+        }
+
+        @Override
+        public MBeanInfo getMBeanInfo() {
+            MBeanAttributeInfo[] attrs = new MBeanAttributeInfo[metrics.size()];
+            int i = 0;
+            for (Map.Entry<String, KafkaMetric> entry : this.metrics.entrySet()) {
+                String attribute = entry.getKey();
+                KafkaMetric metric = entry.getValue();
+                attrs[i] = new MBeanAttributeInfo(attribute,
+                                                  double.class.getName(),
+                                                  metric.metricName().description(),
+                                                  true,
+                                                  false,
+                                                  false);
+                i += 1;
+            }
+            return new MBeanInfo(this.getClass().getName(), "", attrs, null, null, null);
+        }
+
+        @Override
+        public Object invoke(String name, Object[] params, String[] sig) throws MBeanException, ReflectionException {
+            throw new UnsupportedOperationException("Set not allowed.");
+        }
+
+        @Override
+        public void setAttribute(Attribute attribute) throws AttributeNotFoundException,
+                InvalidAttributeValueException,
+                MBeanException,
+                ReflectionException {
+            throw new UnsupportedOperationException("Set not allowed.");
+        }
+
+        @Override
+        public AttributeList setAttributes(AttributeList list) {
+            throw new UnsupportedOperationException("Set not allowed.");
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/KafkaMetric.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/KafkaMetric.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/KafkaMetric.java
new file mode 100644
index 0000000..6245e79
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/KafkaMetric.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.kafka_backport.common.metrics;
+
+import org.apache.flink.kafka_backport.common.Metric;
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.apache.flink.kafka_backport.common.utils.Time;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public final class KafkaMetric implements Metric {
+
+    private MetricName metricName;
+    private final Object lock;
+    private final Time time;
+    private final Measurable measurable;
+    private MetricConfig config;
+
+    KafkaMetric(Object lock, MetricName metricName, Measurable measurable, MetricConfig config, Time time) {
+        super();
+        this.metricName = metricName;
+        this.lock = lock;
+        this.measurable = measurable;
+        this.config = config;
+        this.time = time;
+    }
+
+    MetricConfig config() {
+        return this.config;
+    }
+
+    @Override
+    public MetricName metricName() {
+        return this.metricName;
+    }
+
+    @Override
+    public double value() {
+        synchronized (this.lock) {
+            return value(time.milliseconds());
+        }
+    }
+
+    double value(long timeMs) {
+        return this.measurable.measure(config, timeMs);
+    }
+
+    public void config(MetricConfig config) {
+        synchronized (lock) {
+            this.config = config;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Measurable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Measurable.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Measurable.java
new file mode 100644
index 0000000..08ed823
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Measurable.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.kafka_backport.common.metrics;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A measurable quantity that can be registered as a metric
+ */
+public interface Measurable {
+
+    /**
+     * Measure this quantity and return the result as a double
+     * @param config The configuration for this metric
+     * @param now The POSIX time in milliseconds the measurement is being taken
+     * @return The measured value
+     */
+    public double measure(MetricConfig config, long now);
+
+}


[43/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/MultiSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/MultiSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/MultiSend.java
deleted file mode 100644
index 7a90171..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/MultiSend.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.kafka_backport.common.network;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.nio.channels.GatheringByteChannel;
-import java.util.Iterator;
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A set of composite sends, sent one after another
- */
-public class MultiSend implements Send {
-
-    private static final Logger log = LoggerFactory.getLogger(MultiSend.class);
-    private String dest;
-    private long totalWritten = 0;
-    private List<Send> sends;
-    private Iterator<Send> sendsIterator;
-    private Send current;
-    private boolean doneSends = false;
-    private long size = 0;
-
-    public MultiSend(String dest, List<Send> sends) {
-        this.dest = dest;
-        this.sends = sends;
-        this.sendsIterator = sends.iterator();
-        nextSendOrDone();
-        for (Send send: sends)
-            this.size += send.size();
-    }
-
-    @Override
-    public long size() {
-        return size;
-    }
-
-    @Override
-    public String destination() {
-        return dest;
-    }
-
-    @Override
-    public boolean completed() {
-        if (doneSends) {
-            if (totalWritten != size)
-                log.error("mismatch in sending bytes over socket; expected: " + size + " actual: " + totalWritten);
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public long writeTo(GatheringByteChannel channel) throws IOException {
-        if (completed())
-            throw new KafkaException("This operation cannot be completed on a complete request.");
-
-        int totalWrittenPerCall = 0;
-        boolean sendComplete = false;
-        do {
-            long written = current.writeTo(channel);
-            totalWritten += written;
-            totalWrittenPerCall += written;
-            sendComplete = current.completed();
-            if (sendComplete)
-                nextSendOrDone();
-        } while (!completed() && sendComplete);
-        if (log.isTraceEnabled())
-            log.trace("Bytes written as part of multisend call : " + totalWrittenPerCall +  "Total bytes written so far : " + totalWritten + "Expected bytes to write : " + size);
-        return totalWrittenPerCall;
-    }
-
-    // update current if there's a next Send, mark sends as done if there isn't
-    private void nextSendOrDone() {
-        if (sendsIterator.hasNext())
-            current = sendsIterator.next();
-        else
-            doneSends = true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkReceive.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkReceive.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkReceive.java
deleted file mode 100644
index c0d5d99..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkReceive.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.network;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.ScatteringByteChannel;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A size delimited Receive that consists of a 4 byte network-ordered size N followed by N bytes of content
- */
-public class NetworkReceive implements Receive {
-
-    public final static String UNKNOWN_SOURCE = "";
-    public final static int UNLIMITED = -1;
-
-    private final String source;
-    private final ByteBuffer size;
-    private final int maxSize;
-    private ByteBuffer buffer;
-
-
-    public NetworkReceive(String source, ByteBuffer buffer) {
-        this.source = source;
-        this.buffer = buffer;
-        this.size = null;
-        this.maxSize = UNLIMITED;
-    }
-
-    public NetworkReceive(String source) {
-        this.source = source;
-        this.size = ByteBuffer.allocate(4);
-        this.buffer = null;
-        this.maxSize = UNLIMITED;
-    }
-
-    public NetworkReceive(int maxSize, String source) {
-        this.source = source;
-        this.size = ByteBuffer.allocate(4);
-        this.buffer = null;
-        this.maxSize = maxSize;
-    }
-
-    public NetworkReceive() {
-        this(UNKNOWN_SOURCE);
-    }
-
-    @Override
-    public String source() {
-        return source;
-    }
-
-    @Override
-    public boolean complete() {
-        return !size.hasRemaining() && !buffer.hasRemaining();
-    }
-
-    public long readFrom(ScatteringByteChannel channel) throws IOException {
-        return readFromReadableChannel(channel);
-    }
-
-    // Need a method to read from ReadableByteChannel because BlockingChannel requires read with timeout
-    // See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work
-    // This can go away after we get rid of BlockingChannel
-    @Deprecated
-    public long readFromReadableChannel(ReadableByteChannel channel) throws IOException {
-        int read = 0;
-        if (size.hasRemaining()) {
-            int bytesRead = channel.read(size);
-            if (bytesRead < 0)
-                throw new EOFException();
-            read += bytesRead;
-            if (!size.hasRemaining()) {
-                size.rewind();
-                int receiveSize = size.getInt();
-                if (receiveSize < 0)
-                    throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + ")");
-                if (maxSize != UNLIMITED && receiveSize > maxSize)
-                    throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + " larger than " + maxSize + ")");
-                this.buffer = ByteBuffer.allocate(receiveSize);
-            }
-        }
-        if (buffer != null) {
-            int bytesRead = channel.read(buffer);
-            if (bytesRead < 0)
-                throw new EOFException();
-            read += bytesRead;
-        }
-
-        return read;
-    }
-
-    public ByteBuffer payload() {
-        return this.buffer;
-    }
-
-    // Used only by BlockingChannel, so we may be able to get rid of this when/if we get rid of BlockingChannel
-    @Deprecated
-    public long readCompletely(ReadableByteChannel channel) throws IOException {
-        int totalRead = 0;
-        while (!complete()) {
-            totalRead += readFromReadableChannel(channel);
-        }
-        return totalRead;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkSend.java
deleted file mode 100644
index 29ce09d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkSend.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.kafka_backport.common.network;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A size delimited Send that consists of a 4 byte network-ordered size N followed by N bytes of content
- */
-public class NetworkSend extends ByteBufferSend {
-
-    public NetworkSend(String destination, ByteBuffer... buffers) {
-        super(destination, sizeDelimit(buffers));
-    }
-
-    private static ByteBuffer[] sizeDelimit(ByteBuffer[] buffers) {
-        int size = 0;
-        for (int i = 0; i < buffers.length; i++)
-            size += buffers[i].remaining();
-        ByteBuffer[] delimited = new ByteBuffer[buffers.length + 1];
-        delimited[0] = ByteBuffer.allocate(4);
-        delimited[0].putInt(size);
-        delimited[0].rewind();
-        System.arraycopy(buffers, 0, delimited, 1, buffers.length);
-        return delimited;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Receive.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Receive.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Receive.java
deleted file mode 100644
index b799e7c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Receive.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.kafka_backport.common.network;
-
-import java.io.IOException;
-import java.nio.channels.ScatteringByteChannel;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This interface models the in-progress reading of data from a channel to a source identified by an integer id
- */
-public interface Receive {
-
-    /**
-     * The numeric id of the source from which we are receiving data.
-     */
-    public String source();
-
-    /**
-     * Are we done receiving data?
-     */
-    public boolean complete();
-
-    /**
-     * Read bytes into this receive from the given channel
-     * @param channel The channel to read from
-     * @return The number of bytes read
-     * @throws IOException If the reading fails
-     */
-    public long readFrom(ScatteringByteChannel channel) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selectable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selectable.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selectable.java
deleted file mode 100644
index 08da141..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selectable.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.kafka_backport.common.network;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * An interface for asynchronous, multi-channel network I/O
- */
-public interface Selectable {
-
-    /**
-     * Begin establishing a socket connection to the given address identified by the given address
-     * @param id The id for this connection
-     * @param address The address to connect to
-     * @param sendBufferSize The send buffer for the socket
-     * @param receiveBufferSize The receive buffer for the socket
-     * @throws IOException If we cannot begin connecting
-     */
-    public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException;
-
-    /**
-     * Begin disconnecting the connection identified by the given id
-     */
-    public void disconnect(String id);
-
-    /**
-     * Wakeup this selector if it is blocked on I/O
-     */
-    public void wakeup();
-
-    /**
-     * Close this selector
-     */
-    public void close();
-
-    /**
-     * Queue the given request for sending in the subsequent {@poll(long)} calls
-     * @param send The request to send
-     */
-    public void send(Send send);
-
-    /**
-     * Do I/O. Reads, writes, connection establishment, etc.
-     * @param timeout The amount of time to block if there is nothing to do
-     * @throws IOException
-     */
-    public void poll(long timeout) throws IOException;
-
-    /**
-     * The list of sends that completed on the last {@link #poll(long, List) poll()} call.
-     */
-    public List<Send> completedSends();
-
-    /**
-     * The list of receives that completed on the last {@link #poll(long, List) poll()} call.
-     */
-    public List<NetworkReceive> completedReceives();
-
-    /**
-     * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()}
-     * call.
-     */
-    public List<String> disconnected();
-
-    /**
-     * The list of connections that completed their connection on the last {@link #poll(long, List) poll()}
-     * call.
-     */
-    public List<String> connected();
-
-    /**
-     * Disable reads from the given connection
-     * @param id The id for the connection
-     */
-    public void mute(String id);
-
-    /**
-     * Re-enable reads from the given connection
-     * @param id The id for the connection
-     */
-    public void unmute(String id);
-
-    /**
-     * Disable reads from all connections
-     */
-    public void muteAll();
-
-    /**
-     * Re-enable reads from all connections
-     */
-    public void unmuteAll();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selector.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selector.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selector.java
deleted file mode 100644
index a886e3b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selector.java
+++ /dev/null
@@ -1,664 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.network;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-import org.apache.flink.kafka_backport.common.metrics.Measurable;
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.apache.flink.kafka_backport.common.metrics.Metrics;
-import org.apache.flink.kafka_backport.common.metrics.Sensor;
-import org.apache.flink.kafka_backport.common.metrics.stats.Avg;
-import org.apache.flink.kafka_backport.common.metrics.stats.Count;
-import org.apache.flink.kafka_backport.common.metrics.stats.Max;
-import org.apache.flink.kafka_backport.common.metrics.stats.Rate;
-import org.apache.flink.kafka_backport.common.utils.SystemTime;
-import org.apache.flink.kafka_backport.common.utils.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.net.ConnectException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.nio.channels.CancelledKeyException;
-import java.nio.channels.ClosedChannelException;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
-import java.nio.channels.UnresolvedAddressException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A nioSelector interface for doing non-blocking multi-connection network I/O.
- * <p>
- * This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and
- * responses.
- * <p>
- * A connection can be added to the nioSelector associated with an integer id by doing
- * 
- * <pre>
- * nioSelector.connect(42, new InetSocketAddress(&quot;google.com&quot;, server.port), 64000, 64000);
- * </pre>
- * 
- * The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating
- * the connection. The successful invocation of this method does not mean a valid connection has been established.
- * 
- * Sending requests, receiving responses, processing connection completions, and disconnections on the existing
- * connections are all done using the <code>poll()</code> call.
- * 
- * <pre>
- * List&lt;NetworkRequest&gt; requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
- * nioSelector.poll(TIMEOUT_MS, requestsToSend);
- * </pre>
- * 
- * The nioSelector maintains several lists that are reset by each call to <code>poll()</code> which are available via
- * various getters. These are reset by each call to <code>poll()</code>.
- * 
- * This class is not thread safe!
- */
-public class Selector implements Selectable {
-
-    private static final Logger log = LoggerFactory.getLogger(Selector.class);
-
-    private final java.nio.channels.Selector nioSelector;
-    private final Map<String, SelectionKey> keys;
-    private final List<Send> completedSends;
-    private final List<NetworkReceive> completedReceives;
-    private final List<String> disconnected;
-    private final List<String> connected;
-    private final List<String> failedSends;
-    private final Time time;
-    private final SelectorMetrics sensors;
-    private final String metricGrpPrefix;
-    private final Map<String, String> metricTags;
-    private final Map<String, Long> lruConnections;
-    private final long connectionsMaxIdleNanos;
-    private final int maxReceiveSize;
-    private final boolean metricsPerConnection;
-    private long currentTimeNanos;
-    private long nextIdleCloseCheckTime;
-
-
-    /**
-     * Create a new nioSelector
-     */
-    public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags, boolean metricsPerConnection) {
-        try {
-            this.nioSelector = java.nio.channels.Selector.open();
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-        this.maxReceiveSize = maxReceiveSize;
-        this.connectionsMaxIdleNanos = connectionMaxIdleMs * 1000 * 1000;
-        this.time = time;
-        this.metricGrpPrefix = metricGrpPrefix;
-        this.metricTags = metricTags;
-        this.keys = new HashMap<String, SelectionKey>();
-        this.completedSends = new ArrayList<Send>();
-        this.completedReceives = new ArrayList<NetworkReceive>();
-        this.connected = new ArrayList<String>();
-        this.disconnected = new ArrayList<String>();
-        this.failedSends = new ArrayList<String>();
-        this.sensors = new SelectorMetrics(metrics);
-        // initial capacity and load factor are default, we set them explicitly because we want to set accessOrder = true
-        this.lruConnections = new LinkedHashMap<String, Long>(16, .75F, true);
-        currentTimeNanos = new SystemTime().nanoseconds();
-        nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos;
-        this.metricsPerConnection = metricsPerConnection;
-    }
-
-    public Selector(long connectionMaxIdleMS, Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags) {
-        this(NetworkReceive.UNLIMITED, connectionMaxIdleMS, metrics, time, metricGrpPrefix, metricTags, true);
-    }
-
-    /**
-     * Begin connecting to the given address and add the connection to this nioSelector associated with the given id
-     * number.
-     * <p>
-     * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)}
-     * call. Check {@link #connected()} to see which (if any) connections have completed after a given poll call.
-     * @param id The id for the new connection
-     * @param address The address to connect to
-     * @param sendBufferSize The send buffer for the new connection
-     * @param receiveBufferSize The receive buffer for the new connection
-     * @throws IllegalStateException if there is already a connection for that id
-     * @throws IOException if DNS resolution fails on the hostname or if the broker is down
-     */
-    @Override
-    public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException {
-        if (this.keys.containsKey(id))
-            throw new IllegalStateException("There is already a connection for id " + id);
-
-        SocketChannel channel = SocketChannel.open();
-        channel.configureBlocking(false);
-        Socket socket = channel.socket();
-        socket.setKeepAlive(true);
-        socket.setSendBufferSize(sendBufferSize);
-        socket.setReceiveBufferSize(receiveBufferSize);
-        socket.setTcpNoDelay(true);
-        try {
-            channel.connect(address);
-        } catch (UnresolvedAddressException e) {
-            channel.close();
-            throw new IOException("Can't resolve address: " + address, e);
-        } catch (IOException e) {
-            channel.close();
-            throw e;
-        }
-        SelectionKey key = channel.register(this.nioSelector, SelectionKey.OP_CONNECT);
-        key.attach(new Transmissions(id));
-        this.keys.put(id, key);
-    }
-
-    /**
-     * Register the nioSelector with an existing channel
-     * Use this on server-side, when a connection is accepted by a different thread but processed by the Selector
-     * Note that we are not checking if the connection id is valid - since the connection already exists
-     */
-    public void register(String id, SocketChannel channel) throws ClosedChannelException {
-        SelectionKey key = channel.register(nioSelector, SelectionKey.OP_READ);
-        key.attach(new Transmissions(id));
-        this.keys.put(id, key);
-    }
-
-    /**
-     * Disconnect any connections for the given id (if there are any). The disconnection is asynchronous and will not be
-     * processed until the next {@link #poll(long, List) poll()} call.
-     */
-    @Override
-    public void disconnect(String id) {
-        SelectionKey key = this.keys.get(id);
-        if (key != null)
-            key.cancel();
-    }
-
-    /**
-     * Interrupt the nioSelector if it is blocked waiting to do I/O.
-     */
-    @Override
-    public void wakeup() {
-        this.nioSelector.wakeup();
-    }
-
-    /**
-     * Close this selector and all associated connections
-     */
-    @Override
-    public void close() {
-        List<String> connections = new LinkedList<String>(keys.keySet());
-        for (String id: connections)
-            close(id);
-
-        try {
-            this.nioSelector.close();
-        } catch (IOException e) {
-            log.error("Exception closing nioSelector:", e);
-        }
-    }
-
-    /**
-     * Queue the given request for sending in the subsequent {@poll(long)} calls
-     * @param send The request to send
-     */
-    public void send(Send send) {
-        SelectionKey key = keyForId(send.destination());
-        Transmissions transmissions = transmissions(key);
-        if (transmissions.hasSend())
-            throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress.");
-        transmissions.send = send;
-        try {
-            key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
-        } catch (CancelledKeyException e) {
-            close(transmissions.id);
-            this.failedSends.add(send.destination());
-        }
-    }
-
-    /**
-     * Do whatever I/O can be done on each connection without blocking. This includes completing connections, completing
-     * disconnections, initiating new sends, or making progress on in-progress sends or receives.
-     * 
-     * When this call is completed the user can check for completed sends, receives, connections or disconnects using
-     * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These
-     * lists will be cleared at the beginning of each {@link #poll(long, List)} call and repopulated by the call if any
-     * completed I/O.
-     * 
-     * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely.
-     * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is
-     *         already an in-progress send
-     */
-    @Override
-    public void poll(long timeout) throws IOException {
-        clear();
-
-        /* check ready keys */
-        long startSelect = time.nanoseconds();
-        int readyKeys = select(timeout);
-        long endSelect = time.nanoseconds();
-        currentTimeNanos = endSelect;
-        this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds());
-
-        if (readyKeys > 0) {
-            Set<SelectionKey> keys = this.nioSelector.selectedKeys();
-            Iterator<SelectionKey> iter = keys.iterator();
-            while (iter.hasNext()) {
-                SelectionKey key = iter.next();
-                iter.remove();
-
-                Transmissions transmissions = transmissions(key);
-                SocketChannel channel = channel(key);
-
-                // register all per-connection metrics at once
-                sensors.maybeRegisterConnectionMetrics(transmissions.id);
-                lruConnections.put(transmissions.id, currentTimeNanos);
-
-                try {
-                    /* complete any connections that have finished their handshake */
-                    if (key.isConnectable()) {
-                        channel.finishConnect();
-                        key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ);
-                        this.connected.add(transmissions.id);
-                        this.sensors.connectionCreated.record();
-                        log.debug("Connection {} created", transmissions.id);
-                    }
-
-                    /* read from any connections that have readable data */
-                    if (key.isReadable()) {
-                        if (!transmissions.hasReceive())
-                            transmissions.receive = new NetworkReceive(maxReceiveSize, transmissions.id);
-                        try {
-                            transmissions.receive.readFrom(channel);
-                        } catch (InvalidReceiveException e) {
-                            log.error("Invalid data received from " + transmissions.id + " closing connection", e);
-                            close(transmissions.id);
-                            this.disconnected.add(transmissions.id);
-                            throw e;
-                        }
-                        if (transmissions.receive.complete()) {
-                            transmissions.receive.payload().rewind();
-                            this.completedReceives.add(transmissions.receive);
-                            this.sensors.recordBytesReceived(transmissions.id, transmissions.receive.payload().limit());
-                            transmissions.clearReceive();
-                        }
-                    }
-
-                    /* write to any sockets that have space in their buffer and for which we have data */
-                    if (key.isWritable()) {
-                        transmissions.send.writeTo(channel);
-                        if (transmissions.send.completed()) {
-                            this.completedSends.add(transmissions.send);
-                            this.sensors.recordBytesSent(transmissions.id, transmissions.send.size());
-                            transmissions.clearSend();
-                            key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
-                        }
-                    }
-
-                    /* cancel any defunct sockets */
-                    if (!key.isValid()) {
-                        close(transmissions.id);
-                        this.disconnected.add(transmissions.id);
-                    }
-                } catch (IOException e) {
-                    String desc = socketDescription(channel);
-                    if (e instanceof EOFException || e instanceof ConnectException)
-                        log.debug("Connection {} disconnected", desc);
-                    else
-                        log.warn("Error in I/O with connection to {}", desc, e);
-                    close(transmissions.id);
-                    this.disconnected.add(transmissions.id);
-                }
-            }
-        }
-        long endIo = time.nanoseconds();
-        this.sensors.ioTime.record(endIo - endSelect, time.milliseconds());
-        maybeCloseOldestConnection();
-    }
-
-    private String socketDescription(SocketChannel channel) {
-        Socket socket = channel.socket();
-        if (socket == null)
-            return "[unconnected socket]";
-        else if (socket.getInetAddress() != null)
-            return socket.getInetAddress().toString();
-        else
-            return socket.getLocalAddress().toString();
-    }
-
-    @Override
-    public List<Send> completedSends() {
-        return this.completedSends;
-    }
-
-    @Override
-    public List<NetworkReceive> completedReceives() {
-        return this.completedReceives;
-    }
-
-    @Override
-    public List<String> disconnected() {
-        return this.disconnected;
-    }
-
-    @Override
-    public List<String> connected() {
-        return this.connected;
-    }
-
-    @Override
-    public void mute(String id) {
-        mute(this.keyForId(id));
-    }
-
-    private void mute(SelectionKey key) {
-        key.interestOps(key.interestOps() & ~SelectionKey.OP_READ);
-    }
-
-    @Override
-    public void unmute(String id) {
-        unmute(this.keyForId(id));
-    }
-
-    private void unmute(SelectionKey key) {
-        key.interestOps(key.interestOps() | SelectionKey.OP_READ);
-    }
-
-    @Override
-    public void muteAll() {
-        for (SelectionKey key : this.keys.values())
-            mute(key);
-    }
-
-    @Override
-    public void unmuteAll() {
-        for (SelectionKey key : this.keys.values())
-            unmute(key);
-    }
-
-    private void maybeCloseOldestConnection() {
-        if (currentTimeNanos > nextIdleCloseCheckTime) {
-            if (lruConnections.isEmpty()) {
-                nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos;
-            } else {
-                Map.Entry<String, Long> oldestConnectionEntry = lruConnections.entrySet().iterator().next();
-                Long connectionLastActiveTime = oldestConnectionEntry.getValue();
-                nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos;
-                if (currentTimeNanos > nextIdleCloseCheckTime) {
-                    String connectionId = oldestConnectionEntry.getKey();
-                    if (log.isTraceEnabled())
-                        log.trace("About to close the idle connection from " + connectionId
-                                + " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis");
-
-                    disconnected.add(connectionId);
-                    close(connectionId);
-                }
-            }
-        }
-    }
-
-    /**
-     * Clear the results from the prior poll
-     */
-    private void clear() {
-        this.completedSends.clear();
-        this.completedReceives.clear();
-        this.connected.clear();
-        this.disconnected.clear();
-        this.disconnected.addAll(this.failedSends);
-        this.failedSends.clear();
-    }
-
-    /**
-     * Check for data, waiting up to the given timeout.
-     * 
-     * @param ms Length of time to wait, in milliseconds. If negative, wait indefinitely.
-     * @return The number of keys ready
-     * @throws IOException
-     */
-    private int select(long ms) throws IOException {
-        if (ms == 0L)
-            return this.nioSelector.selectNow();
-        else if (ms < 0L)
-            return this.nioSelector.select();
-        else
-            return this.nioSelector.select(ms);
-    }
-
-    /**
-     * Begin closing this connection
-     */
-    public void close(String id) {
-        SelectionKey key = keyForId(id);
-        lruConnections.remove(id);
-        SocketChannel channel = channel(key);
-        Transmissions trans = transmissions(key);
-        if (trans != null) {
-            this.keys.remove(trans.id);
-            trans.clearReceive();
-            trans.clearSend();
-        }
-        key.attach(null);
-        key.cancel();
-        try {
-            channel.socket().close();
-            channel.close();
-        } catch (IOException e) {
-            log.error("Exception closing connection to node {}:", trans.id, e);
-        }
-        this.sensors.connectionClosed.record();
-    }
-
-    /**
-     * Get the selection key associated with this numeric id
-     */
-    private SelectionKey keyForId(String id) {
-        SelectionKey key = this.keys.get(id);
-        if (key == null)
-            throw new IllegalStateException("Attempt to write to socket for which there is no open connection. Connection id " + id + " existing connections " + keys.keySet().toString());
-        return key;
-    }
-
-    /**
-     * Get the transmissions for the given connection
-     */
-    private Transmissions transmissions(SelectionKey key) {
-        return (Transmissions) key.attachment();
-    }
-
-    /**
-     * Get the socket channel associated with this selection key
-     */
-    private SocketChannel channel(SelectionKey key) {
-        return (SocketChannel) key.channel();
-    }
-
-    /**
-     * The id and in-progress send and receive associated with a connection
-     */
-    private static class Transmissions {
-        public String id;
-        public Send send;
-        public NetworkReceive receive;
-
-        public Transmissions(String id) {
-            this.id = id;
-        }
-
-        public boolean hasSend() {
-            return this.send != null;
-        }
-
-        public void clearSend() {
-            this.send = null;
-        }
-
-        public boolean hasReceive() {
-            return this.receive != null;
-        }
-
-        public void clearReceive() {
-            this.receive = null;
-        }
-    }
-
-    private class SelectorMetrics {
-        private final Metrics metrics;
-        public final Sensor connectionClosed;
-        public final Sensor connectionCreated;
-        public final Sensor bytesTransferred;
-        public final Sensor bytesSent;
-        public final Sensor bytesReceived;
-        public final Sensor selectTime;
-        public final Sensor ioTime;
-
-        public SelectorMetrics(Metrics metrics) {
-            this.metrics = metrics;
-            String metricGrpName = metricGrpPrefix + "-metrics";
-            StringBuilder tagsSuffix = new StringBuilder();
-
-            for (Map.Entry<String, String> tag: metricTags.entrySet()) {
-                tagsSuffix.append(tag.getKey());
-                tagsSuffix.append("-");
-                tagsSuffix.append(tag.getValue());
-            }
-
-            this.connectionClosed = this.metrics.sensor("connections-closed:" + tagsSuffix.toString());
-            MetricName metricName = new MetricName("connection-close-rate", metricGrpName, "Connections closed per second in the window.", metricTags);
-            this.connectionClosed.add(metricName, new Rate());
-
-            this.connectionCreated = this.metrics.sensor("connections-created:" + tagsSuffix.toString());
-            metricName = new MetricName("connection-creation-rate", metricGrpName, "New connections established per second in the window.", metricTags);
-            this.connectionCreated.add(metricName, new Rate());
-
-            this.bytesTransferred = this.metrics.sensor("bytes-sent-received:" + tagsSuffix.toString());
-            metricName = new MetricName("network-io-rate", metricGrpName, "The average number of network operations (reads or writes) on all connections per second.", metricTags);
-            bytesTransferred.add(metricName, new Rate(new Count()));
-
-            this.bytesSent = this.metrics.sensor("bytes-sent:" + tagsSuffix.toString(), bytesTransferred);
-            metricName = new MetricName("outgoing-byte-rate", metricGrpName, "The average number of outgoing bytes sent per second to all servers.", metricTags);
-            this.bytesSent.add(metricName, new Rate());
-            metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags);
-            this.bytesSent.add(metricName, new Rate(new Count()));
-            metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", metricTags);
-            this.bytesSent.add(metricName, new Avg());
-            metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", metricTags);
-            this.bytesSent.add(metricName, new Max());
-
-            this.bytesReceived = this.metrics.sensor("bytes-received:" + tagsSuffix.toString(), bytesTransferred);
-            metricName = new MetricName("incoming-byte-rate", metricGrpName, "Bytes/second read off all sockets", metricTags);
-            this.bytesReceived.add(metricName, new Rate());
-            metricName = new MetricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags);
-            this.bytesReceived.add(metricName, new Rate(new Count()));
-
-            this.selectTime = this.metrics.sensor("select-time:" + tagsSuffix.toString());
-            metricName = new MetricName("select-rate", metricGrpName, "Number of times the I/O layer checked for new I/O to perform per second", metricTags);
-            this.selectTime.add(metricName, new Rate(new Count()));
-            metricName = new MetricName("io-wait-time-ns-avg", metricGrpName, "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", metricTags);
-            this.selectTime.add(metricName, new Avg());
-            metricName = new MetricName("io-wait-ratio", metricGrpName, "The fraction of time the I/O thread spent waiting.", metricTags);
-            this.selectTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
-
-            this.ioTime = this.metrics.sensor("io-time:" + tagsSuffix.toString());
-            metricName = new MetricName("io-time-ns-avg", metricGrpName, "The average length of time for I/O per select call in nanoseconds.", metricTags);
-            this.ioTime.add(metricName, new Avg());
-            metricName = new MetricName("io-ratio", metricGrpName, "The fraction of time the I/O thread spent doing I/O", metricTags);
-            this.ioTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
-
-            metricName = new MetricName("connection-count", metricGrpName, "The current number of active connections.", metricTags);
-            this.metrics.addMetric(metricName, new Measurable() {
-                public double measure(MetricConfig config, long now) {
-                    return keys.size();
-                }
-            });
-        }
-
-        public void maybeRegisterConnectionMetrics(String connectionId) {
-            if (!connectionId.isEmpty() && metricsPerConnection) {
-                // if one sensor of the metrics has been registered for the connection,
-                // then all other sensors should have been registered; and vice versa
-                String nodeRequestName = "node-" + connectionId + ".bytes-sent";
-                Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
-                if (nodeRequest == null) {
-                    String metricGrpName = metricGrpPrefix + "-node-metrics";
-
-                    Map<String, String> tags = new LinkedHashMap<String, String>(metricTags);
-                    tags.put("node-id", "node-" + connectionId);
-
-                    nodeRequest = this.metrics.sensor(nodeRequestName);
-                    MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags);
-                    nodeRequest.add(metricName, new Rate());
-                    metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", tags);
-                    nodeRequest.add(metricName, new Rate(new Count()));
-                    metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", tags);
-                    nodeRequest.add(metricName, new Avg());
-                    metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", tags);
-                    nodeRequest.add(metricName, new Max());
-
-                    String nodeResponseName = "node-" + connectionId + ".bytes-received";
-                    Sensor nodeResponse = this.metrics.sensor(nodeResponseName);
-                    metricName = new MetricName("incoming-byte-rate", metricGrpName, tags);
-                    nodeResponse.add(metricName, new Rate());
-                    metricName = new MetricName("response-rate", metricGrpName, "The average number of responses received per second.", tags);
-                    nodeResponse.add(metricName, new Rate(new Count()));
-
-                    String nodeTimeName = "node-" + connectionId + ".latency";
-                    Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName);
-                    metricName = new MetricName("request-latency-avg", metricGrpName, tags);
-                    nodeRequestTime.add(metricName, new Avg());
-                    metricName = new MetricName("request-latency-max", metricGrpName, tags);
-                    nodeRequestTime.add(metricName, new Max());
-                }
-            }
-        }
-
-        public void recordBytesSent(String connectionId, long bytes) {
-            long now = time.milliseconds();
-            this.bytesSent.record(bytes, now);
-            if (!connectionId.isEmpty()) {
-                String nodeRequestName = "node-" + connectionId + ".bytes-sent";
-                Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
-                if (nodeRequest != null)
-                    nodeRequest.record(bytes, now);
-            }
-        }
-
-        public void recordBytesReceived(String connection, int bytes) {
-            long now = time.milliseconds();
-            this.bytesReceived.record(bytes, now);
-            if (!connection.isEmpty()) {
-                String nodeRequestName = "node-" + connection + ".bytes-received";
-                Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
-                if (nodeRequest != null)
-                    nodeRequest.record(bytes, now);
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Send.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Send.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Send.java
deleted file mode 100644
index b9e8a50..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Send.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.kafka_backport.common.network;
-
-import java.io.IOException;
-import java.nio.channels.GatheringByteChannel;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This interface models the in-progress sending of data to a destination identified by an integer id.
- */
-public interface Send {
-
-    /**
-     * The numeric id for the destination of this send
-     */
-    public String destination();
-
-    /**
-     * Is this send complete?
-     */
-    public boolean completed();
-
-    /**
-     * Write some as-yet unwritten bytes from this send to the provided channel. It may take multiple calls for the send
-     * to be completely written
-     * @param channel The channel to write to
-     * @return The number of bytes written
-     * @throws IOException If the write fails
-     */
-    public long writeTo(GatheringByteChannel channel) throws IOException;
-
-    /**
-     * Size of the send
-     */
-    public long size();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ApiKeys.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ApiKeys.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ApiKeys.java
deleted file mode 100644
index e12261c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ApiKeys.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.kafka_backport.common.protocol;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Identifiers for all the Kafka APIs
- */
-public enum ApiKeys {
-    PRODUCE(0, "Produce"),
-    FETCH(1, "Fetch"),
-    LIST_OFFSETS(2, "Offsets"),
-    METADATA(3, "Metadata"),
-    LEADER_AND_ISR(4, "LeaderAndIsr"),
-    STOP_REPLICA(5, "StopReplica"),
-    UPDATE_METADATA_KEY(6, "UpdateMetadata"),
-    CONTROLLED_SHUTDOWN_KEY(7, "ControlledShutdown"),
-    OFFSET_COMMIT(8, "OffsetCommit"),
-    OFFSET_FETCH(9, "OffsetFetch"),
-    CONSUMER_METADATA(10, "ConsumerMetadata"),
-    JOIN_GROUP(11, "JoinGroup"),
-    HEARTBEAT(12, "Heartbeat");
-
-    private static ApiKeys[] codeToType;
-    public static final int MAX_API_KEY;
-
-    static {
-        int maxKey = -1;
-        for (ApiKeys key : ApiKeys.values()) {
-            maxKey = Math.max(maxKey, key.id);
-        }
-        codeToType = new ApiKeys[maxKey + 1];
-        for (ApiKeys key : ApiKeys.values()) {
-            codeToType[key.id] = key;
-        }
-        MAX_API_KEY = maxKey;
-    }
-
-    /** the perminant and immutable id of an API--this can't change ever */
-    public final short id;
-
-    /** an english description of the api--this is for debugging and can change */
-    public final String name;
-
-    private ApiKeys(int id, String name) {
-        this.id = (short) id;
-        this.name = name;
-    }
-
-    public static ApiKeys forId(int id) {
-        return codeToType[id];
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Errors.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Errors.java
deleted file mode 100644
index 5ef3b24..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Errors.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.protocol;
-
-import org.apache.flink.kafka_backport.common.errors.ApiException;
-import org.apache.flink.kafka_backport.common.errors.IllegalGenerationException;
-import org.apache.flink.kafka_backport.common.errors.InvalidRequiredAcksException;
-import org.apache.flink.kafka_backport.common.errors.InvalidTopicException;
-import org.apache.flink.kafka_backport.common.errors.LeaderNotAvailableException;
-import org.apache.flink.kafka_backport.common.errors.NetworkException;
-import org.apache.flink.kafka_backport.common.errors.NotCoordinatorForConsumerException;
-import org.apache.flink.kafka_backport.common.errors.NotEnoughReplicasException;
-import org.apache.flink.kafka_backport.common.errors.NotLeaderForPartitionException;
-import org.apache.flink.kafka_backport.common.errors.OffsetLoadInProgressException;
-import org.apache.flink.kafka_backport.common.errors.OffsetMetadataTooLarge;
-import org.apache.flink.kafka_backport.common.errors.OffsetOutOfRangeException;
-import org.apache.flink.kafka_backport.common.errors.RecordBatchTooLargeException;
-import org.apache.flink.kafka_backport.common.errors.TimeoutException;
-import org.apache.flink.kafka_backport.common.errors.UnknownConsumerIdException;
-import org.apache.flink.kafka_backport.common.errors.UnknownServerException;
-import org.apache.flink.kafka_backport.common.errors.UnknownTopicOrPartitionException;
-import org.apache.flink.kafka_backport.common.errors.ConsumerCoordinatorNotAvailableException;
-import org.apache.flink.kafka_backport.common.errors.CorruptRecordException;
-import org.apache.flink.kafka_backport.common.errors.NotEnoughReplicasAfterAppendException;
-import org.apache.flink.kafka_backport.common.errors.RecordTooLargeException;
-
-import java.util.HashMap;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This class contains all the client-server errors--those errors that must be sent from the server to the client. These
- * are thus part of the protocol. The names can be changed but the error code cannot.
- * 
- * Do not add exceptions that occur only on the client or only on the server here.
- */
-public enum Errors {
-    UNKNOWN(-1, new UnknownServerException("The server experienced an unexpected error when processing the request")),
-    NONE(0, null),
-    OFFSET_OUT_OF_RANGE(1,
-            new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")),
-    CORRUPT_MESSAGE(2,
-            new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")),
-    UNKNOWN_TOPIC_OR_PARTITION(3,
-            new UnknownTopicOrPartitionException("This server does not host this topic-partition.")),
-    // TODO: errorCode 4 for InvalidFetchSize
-    LEADER_NOT_AVAILABLE(5,
-            new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")),
-    NOT_LEADER_FOR_PARTITION(6,
-            new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")),
-    REQUEST_TIMED_OUT(7,
-            new TimeoutException("The request timed out.")),
-    // TODO: errorCode 8 for BrokerNotAvailable
-    REPLICA_NOT_AVAILABLE(9,
-            new ApiException("The replica is not available for the requested topic-partition")),
-    MESSAGE_TOO_LARGE(10,
-            new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")),
-    OFFSET_METADATA_TOO_LARGE(12,
-            new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")),
-    NETWORK_EXCEPTION(13,
-            new NetworkException("The server disconnected before a response was received.")),
-    OFFSET_LOAD_IN_PROGRESS(14,
-            new OffsetLoadInProgressException("The coordinator is loading offsets and can't process requests.")),
-    CONSUMER_COORDINATOR_NOT_AVAILABLE(15,
-            new ConsumerCoordinatorNotAvailableException("The coordinator is not available.")),
-    NOT_COORDINATOR_FOR_CONSUMER(16,
-            new NotCoordinatorForConsumerException("This is not the correct coordinator for this consumer.")),
-    INVALID_TOPIC_EXCEPTION(17,
-            new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")),
-    RECORD_LIST_TOO_LARGE(18,
-            new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")),
-    NOT_ENOUGH_REPLICAS(19,
-            new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")),
-    NOT_ENOUGH_REPLICAS_AFTER_APPEND(20,
-            new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")),
-    INVALID_REQUIRED_ACKS(21,
-            new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")),
-    ILLEGAL_GENERATION(22,
-            new IllegalGenerationException("Specified consumer generation id is not valid.")),
-    INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23,
-            new ApiException("The request partition assignment strategy does not match that of the group.")),
-    UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24,
-            new ApiException("The request partition assignment strategy is unknown to the broker.")),
-    UNKNOWN_CONSUMER_ID(25,
-            new UnknownConsumerIdException("The coordinator is not aware of this consumer.")),
-    INVALID_SESSION_TIMEOUT(26,
-            new ApiException("The session timeout is not within an acceptable range.")),
-    COMMITTING_PARTITIONS_NOT_ASSIGNED(27,
-            new ApiException("Some of the committing partitions are not assigned the committer")),
-    INVALID_COMMIT_OFFSET_SIZE(28,
-            new ApiException("The committing offset data size is not valid"));
-
-    private static Map<Class<?>, Errors> classToError = new HashMap<Class<?>, Errors>();
-    private static Map<Short, Errors> codeToError = new HashMap<Short, Errors>();
-
-    static {
-        for (Errors error : Errors.values()) {
-            codeToError.put(error.code(), error);
-            if (error.exception != null)
-                classToError.put(error.exception.getClass(), error);
-        }
-    }
-
-    private final short code;
-    private final ApiException exception;
-
-    private Errors(int code, ApiException exception) {
-        this.code = (short) code;
-        this.exception = exception;
-    }
-
-    /**
-     * An instance of the exception
-     */
-    public ApiException exception() {
-        return this.exception;
-    }
-
-    /**
-     * The error code for the exception
-     */
-    public short code() {
-        return this.code;
-    }
-
-    /**
-     * Throw the exception corresponding to this error if there is one
-     */
-    public void maybeThrow() {
-        if (exception != null) {
-            throw this.exception;
-        }
-    }
-
-    /**
-     * Throw the exception if there is one
-     */
-    public static Errors forCode(short code) {
-        Errors error = codeToError.get(code);
-        return error == null ? UNKNOWN : error;
-    }
-
-    /**
-     * Return the error instance associated with this exception (or UKNOWN if there is none)
-     */
-    public static Errors forException(Throwable t) {
-        Errors error = classToError.get(t.getClass());
-        return error == null ? UNKNOWN : error;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ProtoUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ProtoUtils.java
deleted file mode 100644
index e5ae9a4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ProtoUtils.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.protocol;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ProtoUtils {
-
-    private static Schema schemaFor(Schema[][] schemas, int apiKey, int version) {
-        if (apiKey < 0 || apiKey > schemas.length)
-            throw new IllegalArgumentException("Invalid api key: " + apiKey);
-        Schema[] versions = schemas[apiKey];
-        if (version < 0 || version > versions.length)
-            throw new IllegalArgumentException("Invalid version for API key " + apiKey + ": " + version);
-        return versions[version];
-    }
-
-    public static short latestVersion(int apiKey) {
-        if (apiKey < 0 || apiKey >= Protocol.CURR_VERSION.length)
-            throw new IllegalArgumentException("Invalid api key: " + apiKey);
-        return Protocol.CURR_VERSION[apiKey];
-    }
-
-    public static Schema requestSchema(int apiKey, int version) {
-        return schemaFor(Protocol.REQUESTS, apiKey, version);
-    }
-
-    public static Schema currentRequestSchema(int apiKey) {
-        return requestSchema(apiKey, latestVersion(apiKey));
-    }
-
-    public static Schema responseSchema(int apiKey, int version) {
-        return schemaFor(Protocol.RESPONSES, apiKey, version);
-    }
-
-    public static Schema currentResponseSchema(int apiKey) {
-        return schemaFor(Protocol.RESPONSES, apiKey, latestVersion(apiKey));
-    }
-
-    public static Struct parseRequest(int apiKey, int version, ByteBuffer buffer) {
-        return (Struct) requestSchema(apiKey, version).read(buffer);
-    }
-
-    public static Struct parseResponse(int apiKey, ByteBuffer buffer) {
-        return (Struct) currentResponseSchema(apiKey).read(buffer);
-    }
-
-}


[11/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/CommonClientConfigs.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/CommonClientConfigs.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/CommonClientConfigs.java
deleted file mode 100644
index da44734..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/CommonClientConfigs.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.kafka.copied.clients;
-
-/**
- * Some configurations shared by both producer and consumer
- */
-public class CommonClientConfigs {
-    
-    /*
-     * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
-     */
-
-    public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
-    public static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping&mdash;this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form "
-                                                       + "<code>host1:port1,host2:port2,...</code>. Since these servers are just used for the initial connection to "
-                                                       + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of "
-                                                       + "servers (you may want more than one, though, in case a server is down).";
-    
-    public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms";
-    public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions.";
-    
-    public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes";
-    public static final String SEND_BUFFER_DOC = "The size of the TCP send buffer (SO_SNDBUF) to use when sending data.";
-
-    public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes";
-    public static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer (SO_RCVBUF) to use when reading data.";
-
-    public static final String CLIENT_ID_CONFIG = "client.id";
-    public static final String CLIENT_ID_DOC = "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging.";
-
-    public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms";
-    public static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all requests sent by the consumer to the broker.";
-
-    public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms";
-    public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop.";
-    
-    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms";
-    public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The number of samples maintained to compute metrics.";
-
-    public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples";
-    public static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics.";
-
-    public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters";
-    public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics.";
-
-    public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = "connections.max.idle.ms";
-    public static final String CONNECTIONS_MAX_IDLE_MS_DOC = "Close idle connections after the number of milliseconds specified by this config.";
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ConnectionState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ConnectionState.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ConnectionState.java
deleted file mode 100644
index 6ff294b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/ConnectionState.java
+++ /dev/null
@@ -1,20 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients;
-
-/**
- * The states of a node connection
- */
-public enum ConnectionState {
-    DISCONNECTED, CONNECTING, CONNECTED
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/InFlightRequests.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/InFlightRequests.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/InFlightRequests.java
deleted file mode 100644
index cb3d38d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/InFlightRequests.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.kafka.copied.clients;
-
-import java.util.ArrayDeque;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The set of requests which have been sent or are being sent but haven't yet received a response
- */
-final class InFlightRequests {
-
-    private final int maxInFlightRequestsPerConnection;
-    private final Map<String, Deque<ClientRequest>> requests = new HashMap<String, Deque<ClientRequest>>();
-
-    public InFlightRequests(int maxInFlightRequestsPerConnection) {
-        this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection;
-    }
-
-    /**
-     * Add the given request to the queue for the connection it was directed to
-     */
-    public void add(ClientRequest request) {
-        Deque<ClientRequest> reqs = this.requests.get(request.request().destination());
-        if (reqs == null) {
-            reqs = new ArrayDeque<ClientRequest>();
-            this.requests.put(request.request().destination(), reqs);
-        }
-        reqs.addFirst(request);
-    }
-
-    /**
-     * Get the request queue for the given node
-     */
-    private Deque<ClientRequest> requestQueue(String node) {
-        Deque<ClientRequest> reqs = requests.get(node);
-        if (reqs == null || reqs.isEmpty())
-            throw new IllegalStateException("Response from server for which there are no in-flight requests.");
-        return reqs;
-    }
-
-    /**
-     * Get the oldest request (the one that that will be completed next) for the given node
-     */
-    public ClientRequest completeNext(String node) {
-        return requestQueue(node).pollLast();
-    }
-
-    /**
-     * Get the last request we sent to the given node (but don't remove it from the queue)
-     * @param node The node id
-     */
-    public ClientRequest lastSent(String node) {
-        return requestQueue(node).peekFirst();
-    }
-
-    /**
-     * Complete the last request that was sent to a particular node.
-     * @param node The node the request was sent to
-     * @return The request
-     */
-    public ClientRequest completeLastSent(String node) {
-        return requestQueue(node).pollFirst();
-    }
-
-    /**
-     * Can we send more requests to this node?
-     * 
-     * @param node Node in question
-     * @return true iff we have no requests still being sent to the given node
-     */
-    public boolean canSendMore(String node) {
-        Deque<ClientRequest> queue = requests.get(node);
-        return queue == null || queue.isEmpty() ||
-               (queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection);
-    }
-
-    /**
-     * Return the number of inflight requests directed at the given node
-     * @param node The node
-     * @return The request count.
-     */
-    public int inFlightRequestCount(String node) {
-        Deque<ClientRequest> queue = requests.get(node);
-        return queue == null ? 0 : queue.size();
-    }
-
-    /**
-     * Count all in-flight requests for all nodes
-     */
-    public int inFlightRequestCount() {
-        int total = 0;
-        for (Deque<ClientRequest> deque : this.requests.values())
-            total += deque.size();
-        return total;
-    }
-
-    /**
-     * Clear out all the in-flight requests for the given node and return them
-     * 
-     * @param node The node
-     * @return All the in-flight requests for that node that have been removed
-     */
-    public Iterable<ClientRequest> clearAll(String node) {
-        Deque<ClientRequest> reqs = requests.get(node);
-        if (reqs == null) {
-            return Collections.emptyList();
-        } else {
-            return requests.remove(node);
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/KafkaClient.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/KafkaClient.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/KafkaClient.java
deleted file mode 100644
index 9c53d0d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/KafkaClient.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients;
-
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.requests.RequestHeader;
-
-import java.io.Closeable;
-import java.util.List;
-
-/**
- * The interface for {@link NetworkClient}
- */
-public interface KafkaClient extends Closeable {
-
-    /**
-     * Check if we are currently ready to send another request to the given node but don't attempt to connect if we
-     * aren't.
-     * 
-     * @param node The node to check
-     * @param now The current timestamp
-     */
-    public boolean isReady(Node node, long now);
-
-    /**
-     * Initiate a connection to the given node (if necessary), and return true if already connected. The readiness of a
-     * node will change only when poll is invoked.
-     * 
-     * @param node The node to connect to.
-     * @param now The current time
-     * @return true iff we are ready to immediately initiate the sending of another request to the given node.
-     */
-    public boolean ready(Node node, long now);
-
-    /**
-     * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
-     * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
-     * connections.
-     * 
-     * @param node The node to check
-     * @param now The current timestamp
-     * @return The number of milliseconds to wait.
-     */
-    public long connectionDelay(Node node, long now);
-
-    /**
-     * Check if the connection of the node has failed, based on the connection state. Such connection failure are
-     * usually transient and can be resumed in the next {@link #ready(Node, long)} }
-     * call, but there are cases where transient failures needs to be caught and re-acted upon.
-     *
-     * @param node the node to check
-     * @return true iff the connection has failed and the node is disconnected
-     */
-    public boolean connectionFailed(Node node);
-
-    /**
-     * Queue up the given request for sending. Requests can only be sent on ready connections.
-     * 
-     * @param request The request
-     */
-    public void send(ClientRequest request);
-
-    /**
-     * Do actual reads and writes from sockets.
-     * 
-     * @param timeout The maximum amount of time to wait for responses in ms
-     * @param now The current time in ms
-     * @throws IllegalStateException If a request is sent to an unready node
-     */
-    public List<ClientResponse> poll(long timeout, long now);
-
-    /**
-     * Complete all in-flight requests for a given connection
-     * 
-     * @param id The connection to complete requests for
-     * @param now The current time in ms
-     * @return All requests that complete during this time period.
-     */
-    public List<ClientResponse> completeAll(String id, long now);
-
-    /**
-     * Complete all in-flight requests
-     * 
-     * @param now The current time in ms
-     * @return All requests that complete during this time period.
-     */
-    public List<ClientResponse> completeAll(long now);
-
-    /**
-     * Choose the node with the fewest outstanding requests. This method will prefer a node with an existing connection,
-     * but will potentially choose a node for which we don't yet have a connection if all existing connections are in
-     * use.
-     * 
-     * @param now The current time in ms
-     * @return The node with the fewest in-flight requests.
-     */
-    public Node leastLoadedNode(long now);
-
-    /**
-     * The number of currently in-flight requests for which we have not yet returned a response
-     */
-    public int inFlightRequestCount();
-
-    /**
-     * Get the total in-flight requests for a particular node
-     * 
-     * @param nodeId The id of the node
-     */
-    public int inFlightRequestCount(String nodeId);
-
-    /**
-     * Generate a request header for the next request
-     * 
-     * @param key The API key of the request
-     */
-    public RequestHeader nextRequestHeader(ApiKeys key);
-
-    /**
-     * Wake up the client if it is currently blocked waiting for I/O
-     */
-    public void wakeup();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/Metadata.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/Metadata.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/Metadata.java
deleted file mode 100644
index 5210ce0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/Metadata.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients;
-
-import org.apache.kafka.copied.common.Cluster;
-import org.apache.kafka.copied.common.errors.TimeoutException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * A class encapsulating some of the logic around metadata.
- * <p>
- * This class is shared by the client thread (for partitioning) and the background sender thread.
- * 
- * Metadata is maintained for only a subset of topics, which can be added to over time. When we request metadata for a
- * topic we don't have any metadata for it will trigger a metadata update.
- */
-public final class Metadata {
-
-    private static final Logger log = LoggerFactory.getLogger(Metadata.class);
-
-    private final long refreshBackoffMs;
-    private final long metadataExpireMs;
-    private int version;
-    private long lastRefreshMs;
-    private long lastSuccessfulRefreshMs;
-    private Cluster cluster;
-    private boolean needUpdate;
-    private final Set<String> topics;
-
-    /**
-     * Create a metadata instance with reasonable defaults
-     */
-    public Metadata() {
-        this(100L, 60 * 60 * 1000L);
-    }
-
-    /**
-     * Create a new Metadata instance
-     * @param refreshBackoffMs The minimum amount of time that must expire between metadata refreshes to avoid busy
-     *        polling
-     * @param metadataExpireMs The maximum amount of time that metadata can be retained without refresh
-     */
-    public Metadata(long refreshBackoffMs, long metadataExpireMs) {
-        this.refreshBackoffMs = refreshBackoffMs;
-        this.metadataExpireMs = metadataExpireMs;
-        this.lastRefreshMs = 0L;
-        this.lastSuccessfulRefreshMs = 0L;
-        this.version = 0;
-        this.cluster = Cluster.empty();
-        this.needUpdate = false;
-        this.topics = new HashSet<String>();
-    }
-
-    /**
-     * Get the current cluster info without blocking
-     */
-    public synchronized Cluster fetch() {
-        return this.cluster;
-    }
-
-    /**
-     * Add the topic to maintain in the metadata
-     */
-    public synchronized void add(String topic) {
-        topics.add(topic);
-    }
-
-    /**
-     * The next time to update the cluster info is the maximum of the time the current info will expire and the time the
-     * current info can be updated (i.e. backoff time has elapsed); If an update has been request then the expiry time
-     * is now
-     */
-    public synchronized long timeToNextUpdate(long nowMs) {
-        long timeToExpire = needUpdate ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0);
-        long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs;
-        return Math.max(timeToExpire, timeToAllowUpdate);
-    }
-
-    /**
-     * Request an update of the current cluster metadata info, return the current version before the update
-     */
-    public synchronized int requestUpdate() {
-        this.needUpdate = true;
-        return this.version;
-    }
-
-    /**
-     * Wait for metadata update until the current version is larger than the last version we know of
-     */
-    public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs) throws InterruptedException {
-        if (maxWaitMs < 0) {
-            throw new IllegalArgumentException("Max time to wait for metadata updates should not be < 0 milli seconds");
-        }
-        long begin = System.currentTimeMillis();
-        long remainingWaitMs = maxWaitMs;
-        while (this.version <= lastVersion) {
-            if (remainingWaitMs != 0)
-                wait(remainingWaitMs);
-            long elapsed = System.currentTimeMillis() - begin;
-            if (elapsed >= maxWaitMs)
-                throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
-            remainingWaitMs = maxWaitMs - elapsed;
-        }
-    }
-
-    /**
-     * Add one or more topics to maintain metadata for
-     */
-    public synchronized void addTopics(String... topics) {
-        for (String topic : topics)
-            this.topics.add(topic);
-        requestUpdate();
-    }
-
-    /**
-     * Get the list of topics we are currently maintaining metadata for
-     */
-    public synchronized Set<String> topics() {
-        return new HashSet<String>(this.topics);
-    }
-
-    /**
-     * Check if a topic is already in the topic set.
-     * @param topic topic to check
-     * @return true if the topic exists, false otherwise
-     */
-    public synchronized boolean containsTopic(String topic) {
-        return this.topics.contains(topic);
-    }
-
-    /**
-     * Update the cluster metadata
-     */
-    public synchronized void update(Cluster cluster, long now) {
-        this.needUpdate = false;
-        this.lastRefreshMs = now;
-        this.lastSuccessfulRefreshMs = now;
-        this.version += 1;
-        this.cluster = cluster;
-        notifyAll();
-        log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster);
-    }
-    
-    /**
-     * Record an attempt to update the metadata that failed. We need to keep track of this
-     * to avoid retrying immediately.
-     */
-    public synchronized void failedUpdate(long now) {
-        this.lastRefreshMs = now;
-    }
-    
-    /**
-     * @return The current metadata version
-     */
-    public synchronized int version() {
-        return this.version;
-    }
-
-    /**
-     * The last time metadata was successfully updated.
-     */
-    public synchronized long lastSuccessfulUpdate() {
-        return this.lastSuccessfulRefreshMs;
-    }
-
-    /**
-     * The metadata refresh backoff in ms
-     */
-    public long refreshBackoff() {
-        return refreshBackoffMs;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/NetworkClient.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/NetworkClient.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/NetworkClient.java
deleted file mode 100644
index 2253ab4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/NetworkClient.java
+++ /dev/null
@@ -1,519 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients;
-
-import org.apache.kafka.copied.common.Cluster;
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.network.NetworkReceive;
-import org.apache.kafka.copied.common.network.Selectable;
-import org.apache.kafka.copied.common.network.Send;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.requests.MetadataRequest;
-import org.apache.kafka.copied.common.requests.MetadataResponse;
-import org.apache.kafka.copied.common.requests.RequestHeader;
-import org.apache.kafka.copied.common.requests.RequestSend;
-import org.apache.kafka.copied.common.requests.ResponseHeader;
-import org.apache.kafka.copied.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-
-/**
- * A network client for asynchronous request/response network i/o. This is an internal class used to implement the
- * user-facing producer and consumer clients.
- * <p>
- * This class is not thread-safe!
- */
-public class NetworkClient implements KafkaClient {
-
-    private static final Logger log = LoggerFactory.getLogger(NetworkClient.class);
-
-    /* the selector used to perform network i/o */
-    private final Selectable selector;
-
-    /* the current cluster metadata */
-    private final Metadata metadata;
-
-    /* the state of each node's connection */
-    private final ClusterConnectionStates connectionStates;
-
-    /* the set of requests currently being sent or awaiting a response */
-    private final InFlightRequests inFlightRequests;
-
-    /* the socket send buffer size in bytes */
-    private final int socketSendBuffer;
-
-    /* the socket receive size buffer in bytes */
-    private final int socketReceiveBuffer;
-
-    /* the client id used to identify this client in requests to the server */
-    private final String clientId;
-
-    /* a random offset to use when choosing nodes to avoid having all nodes choose the same node */
-    private final int nodeIndexOffset;
-
-    /* the current correlation id to use when sending requests to servers */
-    private int correlation;
-
-    /* true iff there is a metadata request that has been sent and for which we have not yet received a response */
-    private boolean metadataFetchInProgress;
-
-    /* the last timestamp when no broker node is available to connect */
-    private long lastNoNodeAvailableMs;
-
-    public NetworkClient(Selectable selector,
-                         Metadata metadata,
-                         String clientId,
-                         int maxInFlightRequestsPerConnection,
-                         long reconnectBackoffMs,
-                         int socketSendBuffer,
-                         int socketReceiveBuffer) {
-        this.selector = selector;
-        this.metadata = metadata;
-        this.clientId = clientId;
-        this.inFlightRequests = new InFlightRequests(maxInFlightRequestsPerConnection);
-        this.connectionStates = new ClusterConnectionStates(reconnectBackoffMs);
-        this.socketSendBuffer = socketSendBuffer;
-        this.socketReceiveBuffer = socketReceiveBuffer;
-        this.correlation = 0;
-        this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE);
-        this.metadataFetchInProgress = false;
-        this.lastNoNodeAvailableMs = 0;
-    }
-
-    /**
-     * Begin connecting to the given node, return true if we are already connected and ready to send to that node.
-     * 
-     * @param node The node to check
-     * @param now The current timestamp
-     * @return True if we are ready to send to the given node
-     */
-    @Override
-    public boolean ready(Node node, long now) {
-        if (isReady(node, now))
-            return true;
-
-        if (connectionStates.canConnect(node.idString(), now))
-            // if we are interested in sending to a node and we don't have a connection to it, initiate one
-            initiateConnect(node, now);
-
-        return false;
-    }
-
-    /**
-     * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
-     * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
-     * connections.
-     * 
-     * @param node The node to check
-     * @param now The current timestamp
-     * @return The number of milliseconds to wait.
-     */
-    @Override
-    public long connectionDelay(Node node, long now) {
-        return connectionStates.connectionDelay(node.idString(), now);
-    }
-
-    /**
-     * Check if the connection of the node has failed, based on the connection state. Such connection failure are
-     * usually transient and can be resumed in the next {@link #ready(Node, long)} }
-     * call, but there are cases where transient failures needs to be caught and re-acted upon.
-     *
-     * @param node the node to check
-     * @return true iff the connection has failed and the node is disconnected
-     */
-    @Override
-    public boolean connectionFailed(Node node) {
-        return connectionStates.connectionState(node.idString()).equals(ConnectionState.DISCONNECTED);
-    }
-
-    /**
-     * Check if the node with the given id is ready to send more requests.
-     * 
-     * @param node The node
-     * @param now The current time in ms
-     * @return true if the node is ready
-     */
-    @Override
-    public boolean isReady(Node node, long now) {
-        String nodeId = node.idString();
-        if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0)
-            // if we need to update our metadata now declare all requests unready to make metadata requests first
-            // priority
-            return false;
-        else
-            // otherwise we are ready if we are connected and can send more requests
-            return isSendable(nodeId);
-    }
-
-    /**
-     * Are we connected and ready and able to send more requests to the given connection?
-     * 
-     * @param node The node
-     */
-    private boolean isSendable(String node) {
-        return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node);
-    }
-
-    /**
-     * Return the state of the connection to the given node
-     * 
-     * @param node The node to check
-     * @return The connection state
-     */
-    public ConnectionState connectionState(String node) {
-        return connectionStates.connectionState(node);
-    }
-
-    /**
-     * Queue up the given request for sending. Requests can only be sent out to ready nodes.
-     * 
-     * @param request The request
-     */
-    @Override
-    public void send(ClientRequest request) {
-        String nodeId = request.request().destination();
-        if (!isSendable(nodeId))
-            throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready.");
-
-        this.inFlightRequests.add(request);
-        selector.send(request.request());
-    }
-
-    /**
-     * Do actual reads and writes to sockets.
-     * 
-     * @param timeout The maximum amount of time to wait (in ms) for responses if there are none immediately
-     * @param now The current time in milliseconds
-     * @return The list of responses received
-     */
-    @Override
-    public List<ClientResponse> poll(long timeout, long now) {
-        // should we update our metadata?
-        long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now);
-        long timeToNextReconnectAttempt = Math.max(this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now, 0);
-        long waitForMetadataFetch = this.metadataFetchInProgress ? Integer.MAX_VALUE : 0;
-        // if there is no node available to connect, back off refreshing metadata
-        long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt),
-                waitForMetadataFetch);
-        if (metadataTimeout == 0)
-            maybeUpdateMetadata(now);
-        // do the I/O
-        try {
-            this.selector.poll(Math.min(timeout, metadataTimeout));
-        } catch (IOException e) {
-            log.error("Unexpected error during I/O in producer network thread", e);
-        }
-
-        // process completed actions
-        List<ClientResponse> responses = new ArrayList<ClientResponse>();
-        handleCompletedSends(responses, now);
-        handleCompletedReceives(responses, now);
-        handleDisconnections(responses, now);
-        handleConnections();
-
-        // invoke callbacks
-        for (ClientResponse response : responses) {
-            if (response.request().hasCallback()) {
-                try {
-                    response.request().callback().onComplete(response);
-                } catch (Exception e) {
-                    log.error("Uncaught error in request completion:", e);
-                }
-            }
-        }
-
-        return responses;
-    }
-
-    /**
-     * Await all the outstanding responses for requests on the given connection
-     * 
-     * @param node The node to block on
-     * @param now The current time in ms
-     * @return All the collected responses
-     */
-    @Override
-    public List<ClientResponse> completeAll(String node, long now) {
-        try {
-            this.selector.muteAll();
-            this.selector.unmute(node);
-            List<ClientResponse> responses = new ArrayList<ClientResponse>();
-            while (inFlightRequestCount(node) > 0)
-                responses.addAll(poll(Integer.MAX_VALUE, now));
-            return responses;
-        } finally {
-            this.selector.unmuteAll();
-        }
-    }
-
-    /**
-     * Wait for all outstanding requests to complete.
-     */
-    @Override
-    public List<ClientResponse> completeAll(long now) {
-        List<ClientResponse> responses = new ArrayList<ClientResponse>();
-        while (inFlightRequestCount() > 0)
-            responses.addAll(poll(Integer.MAX_VALUE, now));
-        return responses;
-    }
-
-    /**
-     * Get the number of in-flight requests
-     */
-    @Override
-    public int inFlightRequestCount() {
-        return this.inFlightRequests.inFlightRequestCount();
-    }
-
-    /**
-     * Get the number of in-flight requests for a given node
-     */
-    @Override
-    public int inFlightRequestCount(String node) {
-        return this.inFlightRequests.inFlightRequestCount(node);
-    }
-
-    /**
-     * Generate a request header for the given API key
-     * 
-     * @param key The api key
-     * @return A request header with the appropriate client id and correlation id
-     */
-    @Override
-    public RequestHeader nextRequestHeader(ApiKeys key) {
-        return new RequestHeader(key.id, clientId, correlation++);
-    }
-
-    /**
-     * Interrupt the client if it is blocked waiting on I/O.
-     */
-    @Override
-    public void wakeup() {
-        this.selector.wakeup();
-    }
-
-    /**
-     * Close the network client
-     */
-    @Override
-    public void close() {
-        this.selector.close();
-    }
-
-    /**
-     * Choose the node with the fewest outstanding requests which is at least eligible for connection. This method will
-     * prefer a node with an existing connection, but will potentially choose a node for which we don't yet have a
-     * connection if all existing connections are in use. This method will never choose a node for which there is no
-     * existing connection and from which we have disconnected within the reconnect backoff period.
-     * 
-     * @return The node with the fewest in-flight requests.
-     */
-    public Node leastLoadedNode(long now) {
-        List<Node> nodes = this.metadata.fetch().nodes();
-        int inflight = Integer.MAX_VALUE;
-        Node found = null;
-        for (int i = 0; i < nodes.size(); i++) {
-            int idx = Utils.abs((this.nodeIndexOffset + i) % nodes.size());
-            Node node = nodes.get(idx);
-            int currInflight = this.inFlightRequests.inFlightRequestCount(node.idString());
-            if (currInflight == 0 && this.connectionStates.isConnected(node.idString())) {
-                // if we find an established connection with no in-flight requests we can stop right away
-                return node;
-            } else if (!this.connectionStates.isBlackedOut(node.idString(), now) && currInflight < inflight) {
-                // otherwise if this is the best we have found so far, record that
-                inflight = currInflight;
-                found = node;
-            }
-        }
-        return found;
-    }
-
-    /**
-     * Handle any completed request send. In particular if no response is expected consider the request complete.
-     * 
-     * @param responses The list of responses to update
-     * @param now The current time
-     */
-    private void handleCompletedSends(List<ClientResponse> responses, long now) {
-        // if no response is expected then when the send is completed, return it
-        for (Send send : this.selector.completedSends()) {
-            ClientRequest request = this.inFlightRequests.lastSent(send.destination());
-            if (!request.expectResponse()) {
-                this.inFlightRequests.completeLastSent(send.destination());
-                responses.add(new ClientResponse(request, now, false, null));
-            }
-        }
-    }
-
-    /**
-     * Handle any completed receives and update the response list with the responses received.
-     * 
-     * @param responses The list of responses to update
-     * @param now The current time
-     */
-    private void handleCompletedReceives(List<ClientResponse> responses, long now) {
-        for (NetworkReceive receive : this.selector.completedReceives()) {
-            String source = receive.source();
-            ClientRequest req = inFlightRequests.completeNext(source);
-            ResponseHeader header = ResponseHeader.parse(receive.payload());
-            short apiKey = req.request().header().apiKey();
-            Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload());
-            correlate(req.request().header(), header);
-            if (apiKey == ApiKeys.METADATA.id) {
-                handleMetadataResponse(req.request().header(), body, now);
-            } else {
-                // need to add body/header to response here
-                responses.add(new ClientResponse(req, now, false, body));
-            }
-        }
-    }
-
-    private void handleMetadataResponse(RequestHeader header, Struct body, long now) {
-        this.metadataFetchInProgress = false;
-        MetadataResponse response = new MetadataResponse(body);
-        Cluster cluster = response.cluster();
-        // check if any topics metadata failed to get updated
-        if (response.errors().size() > 0) {
-            log.warn("Error while fetching metadata with correlation id {} : {}", header.correlationId(), response.errors());
-        }
-        // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being
-        // created which means we will get errors and no nodes until it exists
-        if (cluster.nodes().size() > 0) {
-            this.metadata.update(cluster, now);
-        } else {
-            log.trace("Ignoring empty metadata response with correlation id {}.", header.correlationId());
-            this.metadata.failedUpdate(now);
-        }
-    }
-
-    /**
-     * Handle any disconnected connections
-     * 
-     * @param responses The list of responses that completed with the disconnection
-     * @param now The current time
-     */
-    private void handleDisconnections(List<ClientResponse> responses, long now) {
-        for (String node : this.selector.disconnected()) {
-            connectionStates.disconnected(node);
-            log.debug("Node {} disconnected.", node);
-            for (ClientRequest request : this.inFlightRequests.clearAll(node)) {
-                log.trace("Cancelled request {} due to node {} being disconnected", request, node);
-                ApiKeys requestKey = ApiKeys.forId(request.request().header().apiKey());
-                if (requestKey == ApiKeys.METADATA)
-                    metadataFetchInProgress = false;
-                else
-                    responses.add(new ClientResponse(request, now, true, null));
-            }
-        }
-        // we got a disconnect so we should probably refresh our metadata and see if that broker is dead
-        if (this.selector.disconnected().size() > 0)
-            this.metadata.requestUpdate();
-    }
-
-    /**
-     * Record any newly completed connections
-     */
-    private void handleConnections() {
-        for (String node : this.selector.connected()) {
-            log.debug("Completed connection to node {}", node);
-            this.connectionStates.connected(node);
-        }
-    }
-
-    /**
-     * Validate that the response corresponds to the request we expect or else explode
-     */
-    private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) {
-        if (requestHeader.correlationId() != responseHeader.correlationId())
-            throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId()
-                    + ") does not match request (" + requestHeader.correlationId() + ")");
-    }
-
-    /**
-     * Create a metadata request for the given topics
-     */
-    private ClientRequest metadataRequest(long now, String node, Set<String> topics) {
-        MetadataRequest metadata = new MetadataRequest(new ArrayList<String>(topics));
-        RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct());
-        return new ClientRequest(now, true, send, null);
-    }
-
-    /**
-     * Add a metadata request to the list of sends if we can make one
-     */
-    private void maybeUpdateMetadata(long now) {
-        // Beware that the behavior of this method and the computation of timeouts for poll() are
-        // highly dependent on the behavior of leastLoadedNode.
-        Node node = this.leastLoadedNode(now);
-        if (node == null) {
-            log.debug("Give up sending metadata request since no node is available");
-            // mark the timestamp for no node available to connect
-            this.lastNoNodeAvailableMs = now;
-            return;
-        }
-        String nodeConnectionId = node.idString();
-
-
-        if (connectionStates.isConnected(nodeConnectionId) && inFlightRequests.canSendMore(nodeConnectionId)) {
-            Set<String> topics = metadata.topics();
-            this.metadataFetchInProgress = true;
-            ClientRequest metadataRequest = metadataRequest(now, nodeConnectionId, topics);
-            log.debug("Sending metadata request {} to node {}", metadataRequest, node.id());
-            this.selector.send(metadataRequest.request());
-            this.inFlightRequests.add(metadataRequest);
-        } else if (connectionStates.canConnect(nodeConnectionId, now)) {
-            // we don't have a connection to this node right now, make one
-            log.debug("Initialize connection to node {} for sending metadata request", node.id());
-            initiateConnect(node, now);
-            // If initiateConnect failed immediately, this node will be put into blackout and we
-            // should allow immediately retrying in case there is another candidate node. If it
-            // is still connecting, the worst case is that we end up setting a longer timeout
-            // on the next round and then wait for the response.
-        } else { // connected, but can't send more OR connecting
-            // In either case, we just need to wait for a network event to let us know the selected
-            // connection might be usable again.
-            this.lastNoNodeAvailableMs = now;
-        }
-    }
-
-    /**
-     * Initiate a connection to the given node
-     */
-    private void initiateConnect(Node node, long now) {
-        String nodeConnectionId = node.idString();
-        try {
-            log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port());
-            this.connectionStates.connecting(nodeConnectionId, now);
-            selector.connect(nodeConnectionId,
-                             new InetSocketAddress(node.host(), node.port()),
-                             this.socketSendBuffer,
-                             this.socketReceiveBuffer);
-        } catch (IOException e) {
-            /* attempt failed, we'll try again after the backoff */
-            connectionStates.disconnected(nodeConnectionId);
-            /* maybe the problem is our metadata, update it */
-            metadata.requestUpdate();
-            log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/RequestCompletionHandler.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/RequestCompletionHandler.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/RequestCompletionHandler.java
deleted file mode 100644
index e698f9f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/RequestCompletionHandler.java
+++ /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.
- */
-package org.apache.kafka.copied.clients;
-
-/**
- * A callback interface for attaching an action to be executed when a request is complete and the corresponding response
- * has been received. This handler will also be invoked if there is a disconnection while handling the request.
- */
-public interface RequestCompletionHandler {
-
-    public void onComplete(ClientResponse response);
-
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/Consumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/Consumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/Consumer.java
deleted file mode 100644
index 3d3e67b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/Consumer.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.kafka.copied.clients.consumer;
-
-import org.apache.kafka.copied.common.Metric;
-import org.apache.kafka.copied.common.MetricName;
-import org.apache.kafka.copied.common.PartitionInfo;
-import org.apache.kafka.copied.common.TopicPartition;
-
-import java.io.Closeable;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * @see KafkaConsumer
- * @see MockConsumer
- */
-public interface Consumer<K, V> extends Closeable {
-    
-    /**
-     * @see KafkaConsumer#subscriptions()
-     */
-    public Set<TopicPartition> subscriptions();
-
-    /**
-     * @see KafkaConsumer#subscribe(String...)
-     */
-    public void subscribe(String... topics);
-
-    /**
-     * @see KafkaConsumer#subscribe(TopicPartition...)
-     */
-    public void subscribe(TopicPartition... partitions);
-
-    /**
-     * @see KafkaConsumer#unsubscribe(String...)
-     */
-    public void unsubscribe(String... topics);
-
-    /**
-     * @see KafkaConsumer#unsubscribe(TopicPartition...)
-     */
-    public void unsubscribe(TopicPartition... partitions);
-
-    /**
-     * @see KafkaConsumer#poll(long)
-     */
-    public ConsumerRecords<K, V> poll(long timeout);
-
-    /**
-     * @see KafkaConsumer#commit(CommitType)
-     */
-    public void commit(CommitType commitType);
-
-    /**
-     * @see KafkaConsumer#commit(CommitType, ConsumerCommitCallback)
-     */
-    public void commit(CommitType commitType, ConsumerCommitCallback callback);
-
-    /**
-     * @see KafkaConsumer#commit(Map, CommitType)
-     */
-    public void commit(Map<TopicPartition, Long> offsets, CommitType commitType);
-
-    /**
-     * @see KafkaConsumer#commit(Map, CommitType, ConsumerCommitCallback)
-     */
-    public void commit(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback);
-
-    /**
-     * @see KafkaConsumer#seek(TopicPartition, long)
-     */
-    public void seek(TopicPartition partition, long offset);
-
-    /**
-     * @see KafkaConsumer#seekToBeginning(TopicPartition...)
-     */
-    public void seekToBeginning(TopicPartition... partitions);
-
-    /**
-     * @see KafkaConsumer#seekToEnd(TopicPartition...)
-     */
-    public void seekToEnd(TopicPartition... partitions);
-
-    /**
-     * @see KafkaConsumer#position(TopicPartition)
-     */
-    public long position(TopicPartition partition);
-
-    /**
-     * @see KafkaConsumer#committed(TopicPartition)
-     */
-    public long committed(TopicPartition partition);
-
-    /**
-     * @see KafkaConsumer#metrics()
-     */
-    public Map<MetricName, ? extends Metric> metrics();
-
-    /**
-     * @see KafkaConsumer#partitionsFor(String)
-     */
-    public List<PartitionInfo> partitionsFor(String topic);
-
-    /**
-     * @see KafkaConsumer#close()
-     */
-    public void close();
-
-    /**
-     * @see KafkaConsumer#wakeup()
-     */
-    public void wakeup();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerCommitCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerCommitCallback.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerCommitCallback.java
deleted file mode 100644
index aa1460c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerCommitCallback.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.kafka.copied.clients.consumer;
-
-import org.apache.kafka.copied.common.TopicPartition;
-
-import java.util.Map;
-
-/**
- * A callback interface that the user can implement to trigger custom actions when a commit request completes. The callback
- * may be executed in any thread calling {@link Consumer#poll(long) poll()}.
- */
-public interface ConsumerCommitCallback {
-
-    /**
-     * A callback method the user can implement to provide asynchronous handling of commit request completion.
-     * This method will be called when the commit request sent to the server has been acknowledged.
-     *
-     * @param offsets A map of the offsets that this callback applies to
-     * @param exception The exception thrown during processing of the request, or null if the commit completed successfully
-     */
-    void onComplete(Map<TopicPartition, Long> offsets, Exception exception);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerConfig.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerConfig.java
deleted file mode 100644
index 0d810fe..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerConfig.java
+++ /dev/null
@@ -1,325 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer;
-
-import org.apache.kafka.copied.clients.CommonClientConfigs;
-import org.apache.kafka.copied.clients.consumer.internals.NoOpConsumerRebalanceCallback;
-import org.apache.kafka.copied.common.config.AbstractConfig;
-import org.apache.kafka.copied.common.config.ConfigDef;
-import org.apache.kafka.copied.common.config.ConfigDef.Importance;
-import org.apache.kafka.copied.common.config.ConfigDef.Type;
-import org.apache.kafka.copied.common.serialization.Deserializer;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.kafka.copied.common.config.ConfigDef.Range.atLeast;
-import static org.apache.kafka.copied.common.config.ConfigDef.ValidString.in;
-
-/**
- * The consumer configuration keys
- */
-public class ConsumerConfig extends AbstractConfig {
-    private static final ConfigDef CONFIG;
-
-    /*
-     * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS
-     * THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
-     */
-
-    /**
-     * <code>group.id</code>
-     */
-    public static final String GROUP_ID_CONFIG = "group.id";
-    private static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using <code>subscribe(topic)</code> or the Kafka-based offset management strategy.";
-
-    /**
-     * <code>session.timeout.ms</code>
-     */
-    public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms";
-    private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities.";
-
-    /**
-     * <code>bootstrap.servers</code>
-     */
-    public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
-
-    /**
-     * <code>enable.auto.commit</code>
-     */
-    public static final String ENABLE_AUTO_COMMIT_CONFIG = "enable.auto.commit";
-    private static final String ENABLE_AUTO_COMMIT_DOC = "If true the consumer's offset will be periodically committed in the background.";
-
-    /**
-     * <code>auto.commit.interval.ms</code>
-     */
-    public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms";
-    private static final String AUTO_COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if <code>enable.auto.commit</code> is set to <code>true</code>.";
-
-    /**
-     * <code>partition.assignment.strategy</code>
-     */
-    public static final String PARTITION_ASSIGNMENT_STRATEGY_CONFIG = "partition.assignment.strategy";
-    private static final String PARTITION_ASSIGNMENT_STRATEGY_DOC = "The friendly name of the partition assignment strategy that the server will use to distribute partition ownership amongst consumer instances when group management is used";
-
-    /**
-     * <code>auto.offset.reset</code>
-     */
-    public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset";
-    private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted): <ul><li>smallest: automatically reset the offset to the smallest offset<li>largest: automatically reset the offset to the largest offset</li><li>none: throw exception to the consumer if no previous offset is found for the consumer's group</li><li>anything else: throw exception to the consumer.</li></ul>";
-
-    /**
-     * <code>fetch.min.bytes</code>
-     */
-    public static final String FETCH_MIN_BYTES_CONFIG = "fetch.min.bytes";
-    private static final String FETCH_MIN_BYTES_DOC = "The minimum amount of data the server should return for a fetch request. If insufficient data is available the request will wait for that much data to accumulate before answering the request. The default setting of 1 byte means that fetch requests are answered as soon as a single byte of data is available or the fetch request times out waiting for data to arrive. Setting this to something greater than 1 will cause the server to wait for larger amounts of data to accumulate which can improve server throughput a bit at the cost of some additional latency.";
-
-    /**
-     * <code>fetch.max.wait.ms</code>
-     */
-    public static final String FETCH_MAX_WAIT_MS_CONFIG = "fetch.max.wait.ms";
-    private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch.min.bytes.";
-
-    /** <code>metadata.max.age.ms</code> */
-    public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG;
-
-    /**
-     * <code>max.partition.fetch.bytes</code>
-     */
-    public static final String MAX_PARTITION_FETCH_BYTES_CONFIG = "max.partition.fetch.bytes";
-    private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server will return. The maximum total memory used for a request will be <code>#partitions * max.partition.fetch.bytes</code>. This size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large message on a certain partition.";
-
-    /** <code>send.buffer.bytes</code> */
-    public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG;
-
-    /** <code>receive.buffer.bytes</code> */
-    public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG;
-
-    /**
-     * <code>client.id</code>
-     */
-    public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
-
-    /**
-     * <code>reconnect.backoff.ms</code>
-     */
-    public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG;
-
-    /**
-     * <code>retry.backoff.ms</code>
-     */
-    public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG;
-
-    /**
-     * <code>metrics.sample.window.ms</code>
-     */
-    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
-
-    /**
-     * <code>metrics.num.samples</code>
-     */
-    public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG;
-
-    /**
-     * <code>metric.reporters</code>
-     */
-    public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG;
-
-    /**
-     * <code>rebalance.callback.class</code>
-     */
-    public static final String CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG = "rebalance.callback.class";
-    private static final String CONSUMER_REBALANCE_CALLBACK_CLASS_DOC = "A user-provided callback to execute when partition assignments change.";
-
-    /**
-     * <code>check.crcs</code>
-     */
-    public static final String CHECK_CRCS_CONFIG = "check.crcs";
-    private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance.";
-    
-    /** <code>key.deserializer</code> */
-    public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer";
-    private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the <code>Deserializer</code> interface.";
-
-    /** <code>value.deserializer</code> */
-    public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer";
-    private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the <code>Deserializer</code> interface.";
-
-    /** <code>connections.max.idle.ms</code> */
-    public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;
-
-
-    static {
-        CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
-                                        Type.LIST,
-                                        Importance.HIGH,
-                                        CommonClientConfigs.BOOSTRAP_SERVERS_DOC)
-                                .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC)
-                                .define(SESSION_TIMEOUT_MS_CONFIG,
-                                        Type.INT,
-                                        30000,
-                                        Importance.HIGH,
-                                        SESSION_TIMEOUT_MS_DOC)
-                                .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
-                                        Type.STRING,
-                                        "range",
-                                        in("range", "roundrobin"),
-                                        Importance.MEDIUM,
-                                        PARTITION_ASSIGNMENT_STRATEGY_DOC)
-                                .define(METADATA_MAX_AGE_CONFIG,
-                                        Type.LONG,
-                                        5 * 60 * 1000,
-                                        atLeast(0),
-                                        Importance.LOW,
-                                        CommonClientConfigs.METADATA_MAX_AGE_DOC)
-                                .define(ENABLE_AUTO_COMMIT_CONFIG,
-                                        Type.BOOLEAN,
-                                        true,
-                                        Importance.MEDIUM,
-                                        ENABLE_AUTO_COMMIT_DOC)
-                                .define(AUTO_COMMIT_INTERVAL_MS_CONFIG,
-                                        Type.LONG,
-                                        5000,
-                                        atLeast(0),
-                                        Importance.LOW,
-                                        AUTO_COMMIT_INTERVAL_MS_DOC)
-                                .define(CLIENT_ID_CONFIG,
-                                        Type.STRING,
-                                        "",
-                                        Importance.LOW,
-                                        CommonClientConfigs.CLIENT_ID_DOC)
-                                .define(MAX_PARTITION_FETCH_BYTES_CONFIG,
-                                        Type.INT,
-                                        1 * 1024 * 1024,
-                                        atLeast(0),
-                                        Importance.HIGH,
-                                        MAX_PARTITION_FETCH_BYTES_DOC)
-                                .define(SEND_BUFFER_CONFIG,
-                                        Type.INT,
-                                        128 * 1024,
-                                        atLeast(0),
-                                        Importance.MEDIUM,
-                                        CommonClientConfigs.SEND_BUFFER_DOC)
-                                .define(RECEIVE_BUFFER_CONFIG,
-                                        Type.INT,
-                                        32 * 1024,
-                                        atLeast(0),
-                                        Importance.MEDIUM,
-                                        CommonClientConfigs.RECEIVE_BUFFER_DOC)
-                                .define(FETCH_MIN_BYTES_CONFIG,
-                                        Type.INT,
-                                        1024,
-                                        atLeast(0),
-                                        Importance.HIGH,
-                                        FETCH_MIN_BYTES_DOC)
-                                .define(FETCH_MAX_WAIT_MS_CONFIG,
-                                        Type.INT,
-                                        500,
-                                        atLeast(0),
-                                        Importance.LOW,
-                                        FETCH_MAX_WAIT_MS_DOC)
-                                .define(RECONNECT_BACKOFF_MS_CONFIG,
-                                        Type.LONG,
-                                        50L,
-                                        atLeast(0L),
-                                        Importance.LOW,
-                                        CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC)
-                                .define(RETRY_BACKOFF_MS_CONFIG,
-                                        Type.LONG,
-                                        100L,
-                                        atLeast(0L),
-                                        Importance.LOW,
-                                        CommonClientConfigs.RETRY_BACKOFF_MS_DOC)
-                                .define(AUTO_OFFSET_RESET_CONFIG,
-                                        Type.STRING,
-                                        "latest",
-                                        in("latest", "earliest", "none"),
-                                        Importance.MEDIUM,
-                                        AUTO_OFFSET_RESET_DOC)
-                                .define(CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
-                                        Type.CLASS,
-                                        NoOpConsumerRebalanceCallback.class,
-                                        Importance.LOW,
-                                        CONSUMER_REBALANCE_CALLBACK_CLASS_DOC)
-                                .define(CHECK_CRCS_CONFIG,
-                                        Type.BOOLEAN,
-                                        true,
-                                        Importance.LOW,
-                                        CHECK_CRCS_DOC)                                
-                                .define(METRICS_SAMPLE_WINDOW_MS_CONFIG,
-                                        Type.LONG,
-                                        30000,
-                                        atLeast(0),
-                                        Importance.LOW,
-                                        CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC)
-                                .define(METRICS_NUM_SAMPLES_CONFIG,
-                                        Type.INT,
-                                        2,
-                                        atLeast(1),
-                                        Importance.LOW,
-                                        CommonClientConfigs.METRICS_NUM_SAMPLES_DOC)
-                                .define(METRIC_REPORTER_CLASSES_CONFIG,
-                                        Type.LIST,
-                                        "",
-                                        Importance.LOW,
-                                        CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC)
-                                .define(KEY_DESERIALIZER_CLASS_CONFIG,
-                                        Type.CLASS,
-                                        Importance.HIGH,
-                                        KEY_DESERIALIZER_CLASS_DOC)
-                                .define(VALUE_DESERIALIZER_CLASS_CONFIG,
-                                        Type.CLASS,
-                                        Importance.HIGH,
-                                        VALUE_DESERIALIZER_CLASS_DOC)
-                                /* default is set to be a bit lower than the server default (10 min), to avoid both client and server closing connection at same time */
-                                .define(CONNECTIONS_MAX_IDLE_MS_CONFIG,
-                                        Type.LONG,
-                                        9 * 60 * 1000,
-                                        Importance.MEDIUM,
-                                        CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC);
-    }
-
-    public static Map<String, Object> addDeserializerToConfig(Map<String, Object> configs,
-                                                              Deserializer<?> keyDeserializer,
-                                                              Deserializer<?> valueDeserializer) {
-        Map<String, Object> newConfigs = new HashMap<String, Object>();
-        newConfigs.putAll(configs);
-        if (keyDeserializer != null)
-            newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass());
-        if (keyDeserializer != null)
-            newConfigs.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass());
-        return newConfigs;
-    }
-
-    public static Properties addDeserializerToConfig(Properties properties,
-                                                     Deserializer<?> keyDeserializer,
-                                                     Deserializer<?> valueDeserializer) {
-        Properties newProperties = new Properties();
-        newProperties.putAll(properties);
-        if (keyDeserializer != null)
-            newProperties.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName());
-        if (keyDeserializer != null)
-            newProperties.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName());
-        return newProperties;
-    }
-
-    public ConsumerConfig(Map<?, ?> props) {
-        super(CONFIG, props);
-    }
-
-    public static void main(String[] args) {
-        System.out.println(CONFIG.toHtmlTable());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRebalanceCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRebalanceCallback.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRebalanceCallback.java
deleted file mode 100644
index 7ab1cd3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/clients/consumer/ConsumerRebalanceCallback.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.clients.consumer;
-
-import org.apache.kafka.copied.common.TopicPartition;
-
-import java.util.Collection;
-
-/**
- * A callback interface that the user can implement to trigger custom actions when the set of partitions assigned to the
- * consumer changes.
- * <p>
- * This is applicable when the consumer is having Kafka auto-manage group membership, if the consumer's directly subscribe to partitions
- * those partitions will never be reassigned and this callback is not applicable.
- * <p>
- * When Kafka is managing the group membership, a partition re-assignment will be triggered any time the members of the group changes or the subscription
- * of the members changes. This can occur when processes die, new process instances are added or old instances come back to life after failure.
- * <p>
- * There are many uses for this functionality. One common use is saving offsets in a custom store. By saving offsets in
- * the {@link #onPartitionsRevoked(Consumer, Collection)} call we can ensure that any time partition assignment changes
- * the offset gets saved.
- * <p>
- * Another use is flushing out any kind of cache of intermediate results the consumer may be keeping. For example,
- * consider a case where the consumer is subscribed to a topic containing user page views, and the goal is to count the
- * number of page views per users for each five minute window. Let's say the topic is partitioned by the user id so that
- * all events for a particular user will go to a single consumer instance. The consumer can keep in memory a running
- * tally of actions per user and only flush these out to a remote data store when it's cache gets to big. However if a
- * partition is reassigned it may want to automatically trigger a flush of this cache, before the new owner takes over
- * consumption.
- * <p>
- * This callback will execute in the user thread as part of the {@link Consumer#poll(long) poll(long)} call whenever partition assignment changes.
- * <p>
- * It is guaranteed that all consumer processes will invoke {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} prior to 
- * any process invoking {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned}. So if offsets or other state is saved in the 
- * {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} call it is guaranteed to be saved by the time the process taking over that
- * partition has their {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned} callback called to load the state.
- * <p>
- * Here is pseudo-code for a callback implementation for saving offsets:
- * <pre>
- * {@code
- *   public class SaveOffsetsOnRebalance implements ConsumerRebalanceCallback {
- *       public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
- *           // read the offsets from an external store using some custom code not described here
- *           for(TopicPartition partition: partitions)
- *              consumer.position(partition, readOffsetFromExternalStore(partition));
- *       }      
- *       public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
- *           // save the offsets in an external store using some custom code not described here
- *           for(TopicPartition partition: partitions)
- *              saveOffsetInExternalStore(consumer.position(partition));
- *       }
- *   }
- * }
- * </pre>
- */
-public interface ConsumerRebalanceCallback {
-
-    /**
-     * A callback method the user can implement to provide handling of customized offsets on completion of a successful
-     * partition re-assignement. This method will be called after an offset re-assignement completes and before the
-     * consumer starts fetching data.
-     * <p>
-     * It is guaranteed that all the processes in a consumer group will execute their
-     * {@link #onPartitionsRevoked(Consumer, Collection)} callback before any instance executes its
-     * {@link #onPartitionsAssigned(Consumer, Collection)} callback.
-     *
-     * @param consumer Reference to the consumer for convenience
-     * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously
-     *            assigned to the consumer)
-     */
-    public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
-
-    /**
-     * A callback method the user can implement to provide handling of offset commits to a customized store on the start
-     * of a rebalance operation. This method will be called before a rebalance operation starts and after the consumer
-     * stops fetching data. It is recommended that offsets should be committed in this callback to either Kafka or a
-     * custom offset store to prevent duplicate data
-     * <p>
-     * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}
-     *
-     * @param consumer  Reference to the consumer for convenience
-     * @param partitions The list of partitions that were assigned to the consumer on the last rebalance
-     */
-    public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
-}


[31/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
index f38c557..863f7ac 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * BarrierBuffer continues receiving buffers from the blocked channels and stores them internally until 
  * the blocks are released.</p>
  */
-public class BarrierBuffer implements CheckpointBarrierHandler, Runnable {
+public class BarrierBuffer implements CheckpointBarrierHandler {
 
 	private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class);
 	
@@ -77,18 +77,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler, Runnable {
 	/** Flag to indicate whether we have drawn all available input */
 	private boolean endOfStream;
 
-
-	private int returnedBuffers;
-	
-	private int spilledBuffers;
-	
-	private int reReadBuffers;
-	
-	
-	private Thread debugPrinter;
-	
-	private volatile boolean printerRunning = true;
-	
 	/**
 	 * 
 	 * @param inputGate The input gate to draw the buffers and events from.
@@ -103,10 +91,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler, Runnable {
 		
 		this.bufferSpiller = new BufferSpiller(ioManager, inputGate.getPageSize());
 		this.queuedBuffered = new ArrayDeque<BufferSpiller.SpilledBufferOrEventSequence>();
-		
-		this.debugPrinter = new Thread(this, "BB debugger");
-		this.debugPrinter.setDaemon(true);
-		this.debugPrinter.start();
 	}
 
 	// ------------------------------------------------------------------------
@@ -127,21 +111,14 @@ public class BarrierBuffer implements CheckpointBarrierHandler, Runnable {
 					completeBufferedSequence();
 					return getNextNonBlocked();
 				}
-				else if (next.isBuffer()) {
-					reReadBuffers++;
-				}
 			}
 			
 			if (next != null) {
 				if (isBlocked(next.getChannelIndex())) {
 					// if the channel is blocked we, we just store the BufferOrEvent
 					bufferSpiller.add(next);
-					if (next.isBuffer()) {
-						spilledBuffers++;
-					}
 				}
 				else if (next.isBuffer()) {
-					returnedBuffers++;
 					return next;
 				}
 				else if (next.getEvent().getClass() == CheckpointBarrier.class) {
@@ -245,9 +222,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler, Runnable {
 
 	@Override
 	public void cleanup() throws IOException {
-		printerRunning = false;
-		debugPrinter.interrupt();
-		
 		bufferSpiller.close();
 		if (currentBuffered != null) {
 			currentBuffered.cleanup();
@@ -343,21 +317,4 @@ public class BarrierBuffer implements CheckpointBarrierHandler, Runnable {
 		return String.format("last checkpoint: %d, current barriers: %d, closed channels: %d",
 				currentCheckpointId, numBarriersReceived, numClosedChannels);
 	}
-	
-	// -------------------------------------
-	// TEMP HACK for debugging
-	
-	public void run() {
-		while (printerRunning) {
-			try {
-				Thread.sleep(5000);
-			}
-			catch (InterruptedException e) {
-				// ignore
-			}
-			
-			LOG.info("=====================> BARRIER BUFFER: returned buffers: {}, spilled buffers: {}, re-read buffers: {}",
-					returnedBuffers, spilledBuffers, reReadBuffers);
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
index 7ab6722..763885c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
@@ -27,12 +27,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
-
 public class ClusterUtil {
+	
 	private static final Logger LOG = LoggerFactory.getLogger(ClusterUtil.class);
 
-	private static LocalFlinkMiniCluster exec = null;
-
 	/**
 	 * Executes the given JobGraph locally, on a FlinkMiniCluster
 	 * 
@@ -42,8 +40,6 @@ public class ClusterUtil {
 	 *            numberOfTaskTrackers
 	 * @param memorySize
 	 *            memorySize
-	 * @param printDuringExecution
-	 * @param detached
 	 * @param customConf
 	 * 		Custom configuration for the LocalExecutor. Can be null.
 	 * @return The result of the job execution, containing elapsed time and accumulators.
@@ -67,7 +63,7 @@ public class ClusterUtil {
 
 		try {
 			exec = new LocalFlinkMiniCluster(configuration, true);
-			if(detached) {
+			if (detached) {
 				exec.submitJobDetached(jobGraph);
 				return null;
 			} else {
@@ -84,17 +80,7 @@ public class ClusterUtil {
 	/**
 	 * Start a job in a detached mode on a local mini cluster.
 	 */
-	public static void startOnMiniCluster(JobGraph jobGraph, int parallelism, long memorySize) throws Exception {
-		runOnMiniCluster(jobGraph, parallelism, memorySize, true, true, null);
-	}
-
-	public static void stopOnMiniCluster() {
-		if(exec != null) {
-			exec.stop();
-			exec = null;
-		} else {
-			throw new IllegalStateException("Cluster was not started via .start(...)");
-		}
+	public static void startOnMiniCluster(JobGraph jobGraph, int parallelism) throws Exception {
+		runOnMiniCluster(jobGraph, parallelism, -1, true, true, null);
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
index 10d4d66..8a75de5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
@@ -31,6 +31,8 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 import java.io.IOException;
@@ -55,6 +57,8 @@ import static org.junit.Assert.assertTrue;
 @SuppressWarnings("serial")
 public class StateCheckpoinedITCase extends StreamFaultToleranceTestBase {
 
+	private static final Logger LOG = LoggerFactory.getLogger(StateCheckpoinedITCase.class);
+
 	final long NUM_STRINGS = 10_000_000L;
 
 	/**
@@ -72,7 +76,9 @@ public class StateCheckpoinedITCase extends StreamFaultToleranceTestBase {
 		final long failurePosMax = (long) (0.7 * NUM_STRINGS / PARALLELISM);
 
 		final long failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
-		
+
+		env.enableCheckpointing(200);
+
 		DataStream<String> stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS));
 
 		stream
@@ -95,8 +101,10 @@ public class StateCheckpoinedITCase extends StreamFaultToleranceTestBase {
 	@Override
 	public void postSubmit() {
 		
-		if (!OnceFailingAggregator.wasCheckpointedBeforeFailure) {
-			System.err.println("Test inconclusive: failure occurred before first checkpoint");
+		//assertTrue("Test inconclusive: failure occurred before first checkpoint",
+		//		OnceFailingAggregator.wasCheckpointedBeforeFailure);
+		if(!OnceFailingAggregator.wasCheckpointedBeforeFailure) {
+			LOG.warn("Test inconclusive: failure occurred before first checkpoint");
 		}
 		
 		long filterSum = 0;

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/tools/maven/suppressions.xml
----------------------------------------------------------------------
diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml
index d76ed47..2c29054 100644
--- a/tools/maven/suppressions.xml
+++ b/tools/maven/suppressions.xml
@@ -25,5 +25,4 @@ under the License.
 <suppressions>
 		<suppress files="org[\\/]apache[\\/]flink[\\/]api[\\/]io[\\/]avro[\\/]example[\\/]User.java" checks="[a-zA-Z0-9]*"/>
 		<suppress files="org[\\/]apache[\\/]flink[\\/]api[\\/]io[\\/]avro[\\/]generated[\\/].*.java" checks="[a-zA-Z0-9]*"/>
-		<suppress files="org[\\/]apache[\\/]flink[\\/]kafka_backport[\\/].*.java" checks="[a-zA-Z0-9]*"/>
 </suppressions>


[48/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/KafkaConsumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/KafkaConsumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/KafkaConsumer.java
deleted file mode 100644
index 8800954..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/KafkaConsumer.java
+++ /dev/null
@@ -1,1130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer;
-
-import org.apache.flink.kafka_backport.clients.Metadata;
-import org.apache.flink.kafka_backport.clients.consumer.internals.DelayedTask;
-import org.apache.flink.kafka_backport.common.KafkaException;
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-import org.apache.flink.kafka_backport.clients.ClientUtils;
-import org.apache.flink.kafka_backport.clients.NetworkClient;
-import org.apache.flink.kafka_backport.clients.consumer.internals.ConsumerNetworkClient;
-import org.apache.flink.kafka_backport.clients.consumer.internals.Coordinator;
-import org.apache.flink.kafka_backport.clients.consumer.internals.Fetcher;
-import org.apache.flink.kafka_backport.clients.consumer.internals.SubscriptionState;
-import org.apache.flink.kafka_backport.common.Cluster;
-import org.apache.flink.kafka_backport.common.Metric;
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.apache.flink.kafka_backport.common.PartitionInfo;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.metrics.JmxReporter;
-import org.apache.flink.kafka_backport.common.metrics.Metrics;
-import org.apache.flink.kafka_backport.common.metrics.MetricsReporter;
-import org.apache.flink.kafka_backport.common.network.Selector;
-import org.apache.flink.kafka_backport.common.serialization.Deserializer;
-import org.apache.flink.kafka_backport.common.utils.SystemTime;
-import org.apache.flink.kafka_backport.common.utils.Time;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetSocketAddress;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.ConcurrentModificationException;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.flink.kafka_backport.common.utils.Utils.min;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A Kafka client that consumes records from a Kafka cluster.
- * <p>
- * It will transparently handle the failure of servers in the Kafka cluster, and transparently adapt as partitions of
- * data it subscribes to migrate within the cluster. This client also interacts with the server to allow groups of
- * consumers to load balance consumption using consumer groups (as described below).
- * <p>
- * The consumer maintains TCP connections to the necessary brokers to fetch data for the topics it subscribes to.
- * Failure to close the consumer after use will leak these connections.
- * <p>
- * The consumer is not thread-safe. See <a href="#multithreaded">Multi-threaded Processing</a> for more details.
- *
- * <h3>Offsets and Consumer Position</h3>
- * Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of
- * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer
- * which has position 5 has consumed records with offsets 0 through 4 and will next receive record with offset 5. There
- * are actually two notions of position relevant to the user of the consumer.
- * <p>
- * The {@link #position(TopicPartition) position} of the consumer gives the offset of the next record that will be given
- * out. It will be one larger than the highest offset the consumer has seen in that partition. It automatically advances
- * every time the consumer receives data calls {@link #poll(long)} and receives messages.
- * <p>
- * The {@link #commit(CommitType) committed position} is the last offset that has been saved securely. Should the
- * process fail and restart, this is the offset that it will recover to. The consumer can either automatically commit
- * offsets periodically, or it can choose to control this committed position manually by calling
- * {@link #commit(CommitType) commit}.
- * <p>
- * This distinction gives the consumer control over when a record is considered consumed. It is discussed in further
- * detail below.
- * 
- * <h3>Consumer Groups</h3>
- * 
- * Kafka uses the concept of <i>consumer groups</i> to allow a pool of processes to divide up the work of consuming and
- * processing records. These processes can either be running on the same machine or, as is more likely, they can be
- * distributed over many machines to provide additional scalability and fault tolerance for processing.
- * <p>
- * Each Kafka consumer must specify a consumer group that it belongs to. Kafka will deliver each message in the
- * subscribed topics to one process in each consumer group. This is achieved by balancing the partitions in the topic
- * over the consumer processes in each group. So if there is a topic with four partitions, and a consumer group with two
- * processes, each process would consume from two partitions. This group membership is maintained dynamically: if a
- * process fails the partitions assigned to it will be reassigned to other processes in the same group, and if a new
- * process joins the group, partitions will be moved from existing consumers to this new process.
- * <p>
- * So if two processes subscribe to a topic both specifying different groups they will each get all the records in that
- * topic; if they both specify the same group they will each get about half the records.
- * <p>
- * Conceptually you can think of a consumer group as being a single logical subscriber that happens to be made up of
- * multiple processes. As a multi-subscriber system, Kafka naturally supports having any number of consumer groups for a
- * given topic without duplicating data (additional consumers are actually quite cheap).
- * <p>
- * This is a slight generalization of the functionality that is common in messaging systems. To get semantics similar to
- * a queue in a traditional messaging system all processes would be part of a single consumer group and hence record
- * delivery would be balanced over the group like with a queue. Unlike a traditional messaging system, though, you can
- * have multiple such groups. To get semantics similar to pub-sub in a traditional messaging system each process would
- * have it's own consumer group, so each process would subscribe to all the records published to the topic.
- * <p>
- * In addition, when offsets are committed they are always committed for a given consumer group.
- * <p>
- * It is also possible for the consumer to manually specify the partitions it subscribes to, which disables this dynamic
- * partition balancing.
- * 
- * <h3>Usage Examples</h3>
- * The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to
- * demonstrate how to use them.
- * 
- * <h4>Simple Processing</h4>
- * This example demonstrates the simplest usage of Kafka's consumer api.
- * 
- * <pre>
- *     Properties props = new Properties();
- *     props.put(&quot;bootstrap.servers&quot;, &quot;localhost:9092&quot;);
- *     props.put(&quot;group.id&quot;, &quot;test&quot;);
- *     props.put(&quot;enable.auto.commit&quot;, &quot;true&quot;);
- *     props.put(&quot;auto.commit.interval.ms&quot;, &quot;1000&quot;);
- *     props.put(&quot;session.timeout.ms&quot;, &quot;30000&quot;);
- *     props.put(&quot;key.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
- *     props.put(&quot;value.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
- *     KafkaConsumer&lt;String, String&gt; consumer = new KafkaConsumer&lt;String, String&gt;(props);
- *     consumer.subscribe(&quot;foo&quot;, &quot;bar&quot;);
- *     while (true) {
- *         ConsumerRecords&lt;String, String&gt; records = consumer.poll(100);
- *         for (ConsumerRecord&lt;String, String&gt; record : records)
- *             System.out.printf(&quot;offset = %d, key = %s, value = %s&quot;, record.offset(), record.key(), record.value());
- *     }
- * </pre>
- * 
- * Setting <code>enable.auto.commit</code> means that offsets are committed automatically with a frequency controlled by
- * the config <code>auto.commit.interval.ms</code>.
- * <p>
- * The connection to the cluster is bootstrapped by specifying a list of one or more brokers to contact using the
- * configuration <code>bootstrap.servers</code>. This list is just used to discover the rest of the brokers in the
- * cluster and need not be an exhaustive list of servers in the cluster (though you may want to specify more than one in
- * case there are servers down when the client is connecting).
- * <p>
- * In this example the client is subscribing to the topics <i>foo</i> and <i>bar</i> as part of a group of consumers
- * called <i>test</i> as described above.
- * <p>
- * The broker will automatically detect failed processes in the <i>test</i> group by using a heartbeat mechanism. The
- * consumer will automatically ping the cluster periodically, which let's the cluster know that it is alive. As long as
- * the consumer is able to do this it is considered alive and retains the right to consume from the partitions assigned
- * to it. If it stops heartbeating for a period of time longer than <code>session.timeout.ms</code> then it will be
- * considered dead and it's partitions will be assigned to another process.
- * <p>
- * The deserializer settings specify how to turn bytes into objects. For example, by specifying string deserializers, we
- * are saying that our record's key and value will just be simple strings.
- * 
- * <h4>Controlling When Messages Are Considered Consumed</h4>
- * 
- * In this example we will consume a batch of records and batch them up in memory, when we have sufficient records
- * batched we will insert them into a database. If we allowed offsets to auto commit as in the previous example messages
- * would be considered consumed after they were given out by the consumer, and it would be possible that our process
- * could fail after we have read messages into our in-memory buffer but before they had been inserted into the database.
- * To avoid this we will manually commit the offsets only once the corresponding messages have been inserted into the
- * database. This gives us exact control of when a message is considered consumed. This raises the opposite possibility:
- * the process could fail in the interval after the insert into the database but before the commit (even though this
- * would likely just be a few milliseconds, it is a possibility). In this case the process that took over consumption
- * would consume from last committed offset and would repeat the insert of the last batch of data. Used in this way
- * Kafka provides what is often called "at-least once delivery" guarantees, as each message will likely be delivered one
- * time but in failure cases could be duplicated.
- * 
- * <pre>
- *     Properties props = new Properties();
- *     props.put(&quot;bootstrap.servers&quot;, &quot;localhost:9092&quot;);
- *     props.put(&quot;group.id&quot;, &quot;test&quot;);
- *     props.put(&quot;enable.auto.commit&quot;, &quot;false&quot;);
- *     props.put(&quot;auto.commit.interval.ms&quot;, &quot;1000&quot;);
- *     props.put(&quot;session.timeout.ms&quot;, &quot;30000&quot;);
- *     props.put(&quot;key.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
- *     props.put(&quot;value.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
- *     KafkaConsumer&lt;String, String&gt; consumer = new KafkaConsumer&lt;String, String&gt;(props);
- *     consumer.subscribe(&quot;foo&quot;, &quot;bar&quot;);
- *     int commitInterval = 200;
- *     List&lt;ConsumerRecord&lt;String, String&gt;&gt; buffer = new ArrayList&lt;ConsumerRecord&lt;String, String&gt;&gt;();
- *     while (true) {
- *         ConsumerRecords&lt;String, String&gt; records = consumer.poll(100);
- *         for (ConsumerRecord&lt;String, String&gt; record : records) {
- *             buffer.add(record);
- *             if (buffer.size() &gt;= commitInterval) {
- *                 insertIntoDb(buffer);
- *                 consumer.commit(CommitType.SYNC);
- *                 buffer.clear();
- *             }
- *         }
- *     }
- * </pre>
- * 
- * <h4>Subscribing To Specific Partitions</h4>
- * 
- * In the previous examples we subscribed to the topics we were interested in and let Kafka give our particular process
- * a fair share of the partitions for those topics. This provides a simple load balancing mechanism so multiple
- * instances of our program can divided up the work of processing records.
- * <p>
- * In this mode the consumer will just get the partitions it subscribes to and if the consumer instance fails no attempt
- * will be made to rebalance partitions to other instances.
- * <p>
- * There are several cases where this makes sense:
- * <ul>
- * <li>The first case is if the process is maintaining some kind of local state associated with that partition (like a
- * local on-disk key-value store) and hence it should only get records for the partition it is maintaining on disk.
- * <li>Another case is if the process itself is highly available and will be restarted if it fails (perhaps using a
- * cluster management framework like YARN, Mesos, or AWS facilities, or as part of a stream processing framework). In
- * this case there is no need for Kafka to detect the failure and reassign the partition, rather the consuming process
- * will be restarted on another machine.
- * </ul>
- * <p>
- * This mode is easy to specify, rather than subscribing to the topic, the consumer just subscribes to particular
- * partitions:
- * 
- * <pre>
- *     String topic = &quot;foo&quot;;
- *     TopicPartition partition0 = new TopicPartition(topic, 0);
- *     TopicPartition partition1 = new TopicPartition(topic, 1);
- *     consumer.subscribe(partition0);
- *     consumer.subscribe(partition1);
- * </pre>
- * 
- * The group that the consumer specifies is still used for committing offsets, but now the set of partitions will only
- * be changed if the consumer specifies new partitions, and no attempt at failure detection will be made.
- * <p>
- * It isn't possible to mix both subscription to specific partitions (with no load balancing) and to topics (with load
- * balancing) using the same consumer instance.
- * 
- * <h4>Managing Your Own Offsets</h4>
- * 
- * The consumer application need not use Kafka's built-in offset storage, it can store offsets in a store of it's own
- * choosing. The primary use case for this is allowing the application to store both the offset and the results of the
- * consumption in the same system in a way that both the results and offsets are stored atomically. This is not always
- * possible, but when it is it will make the consumption fully atomic and give "exactly once" semantics that are
- * stronger than the default "at-least once" semantics you get with Kafka's offset commit functionality.
- * <p>
- * Here are a couple of examples of this type of usage:
- * <ul>
- * <li>If the results of the consumption are being stored in a relational database, storing the offset in the database
- * as well can allow committing both the results and offset in a single transaction. Thus either the transaction will
- * succeed and the offset will be updated based on what was consumed or the result will not be stored and the offset
- * won't be updated.
- * <li>If the results are being stored in a local store it may be possible to store the offset there as well. For
- * example a search index could be built by subscribing to a particular partition and storing both the offset and the
- * indexed data together. If this is done in a way that is atomic, it is often possible to have it be the case that even
- * if a crash occurs that causes unsync'd data to be lost, whatever is left has the corresponding offset stored as well.
- * This means that in this case the indexing process that comes back having lost recent updates just resumes indexing
- * from what it has ensuring that no updates are lost.
- * </ul>
- * 
- * Each record comes with it's own offset, so to manage your own offset you just need to do the following:
- * <ol>
- * <li>Configure <code>enable.auto.commit=false</code>
- * <li>Use the offset provided with each {@link ConsumerRecord} to save your position.
- * <li>On restart restore the position of the consumer using {@link #seek(TopicPartition, long)}.
- * </ol>
- * 
- * This type of usage is simplest when the partition assignment is also done manually (this would be likely in the
- * search index use case described above). If the partition assignment is done automatically special care will also be
- * needed to handle the case where partition assignments change. This can be handled using a special callback specified
- * using <code>rebalance.callback.class</code>, which specifies an implementation of the interface
- * {@link ConsumerRebalanceCallback}. When partitions are taken from a consumer the consumer will want to commit its
- * offset for those partitions by implementing
- * {@link ConsumerRebalanceCallback#onPartitionsRevoked(Consumer, Collection)}. When partitions are assigned to a
- * consumer, the consumer will want to look up the offset for those new partitions an correctly initialize the consumer
- * to that position by implementing {@link ConsumerRebalanceCallback#onPartitionsAssigned(Consumer, Collection)}.
- * <p>
- * Another common use for {@link ConsumerRebalanceCallback} is to flush any caches the application maintains for
- * partitions that are moved elsewhere.
- * 
- * <h4>Controlling The Consumer's Position</h4>
- * 
- * In most use cases the consumer will simply consume records from beginning to end, periodically committing it's
- * position (either automatically or manually). However Kafka allows the consumer to manually control it's position,
- * moving forward or backwards in a partition at will. This means a consumer can re-consume older records, or skip to
- * the most recent records without actually consuming the intermediate records.
- * <p>
- * There are several instances where manually controlling the consumer's position can be useful.
- * <p>
- * One case is for time-sensitive record processing it may make sense for a consumer that falls far enough behind to not
- * attempt to catch up processing all records, but rather just skip to the most recent records.
- * <p>
- * Another use case is for a system that maintains local state as described in the previous section. In such a system
- * the consumer will want to initialize it's position on start-up to whatever is contained in the local store. Likewise
- * if the local state is destroyed (say because the disk is lost) the state may be recreated on a new machine by
- * reconsuming all the data and recreating the state (assuming that Kafka is retaining sufficient history).
- * 
- * Kafka allows specifying the position using {@link #seek(TopicPartition, long)} to specify the new position. Special
- * methods for seeking to the earliest and latest offset the server maintains are also available (
- * {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively).
- * 
- *
- * <h3><a name="multithreaded">Multi-threaded Processing</a></h3>
- * 
- * The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application
- * making the call. It is the responsibility of the user to ensure that multi-threaded access
- * is properly synchronized. Un-synchronized access will result in {@link ConcurrentModificationException}.
- *
- * <p>
- * The only exception to this rule is {@link #wakeup()}, which can safely be used from an external thread to
- * interrupt an active operation. In this case, a {@link ConsumerWakeupException} will be thrown from the thread
- * blocking on the operation. This can be used to shutdown the consumer from another thread. The following
- * snippet shows the typical pattern:
- *
- * <pre>
- * public class KafkaConsumerRunner implements Runnable {
- *     private final AtomicBoolean closed = new AtomicBoolean(false);
- *     private final KafkaConsumer consumer;
- *
- *     public void run() {
- *         try {
- *             consumer.subscribe("topic");
- *             while (!closed.get()) {
- *                 ConsumerRecords records = consumer.poll(10000);
- *                 // Handle new records
- *             }
- *         } catch (ConsumerWakeupException e) {
- *             // Ignore exception if closing
- *             if (!closed.get()) throw e;
- *         } finally {
- *             consumer.close();
- *         }
- *     }
- *
- *     // Shutdown hook which can be called from a separate thread
- *     public void shutdown() {
- *         closed.set(true);
- *         consumer.wakeup();
- *     }
- * }
- * </pre>
- *
- * Then in a separate thread, the consumer can be shutdown by setting the closed flag and waking up the consumer.
- *
- * <pre>
- *     closed.set(true);
- *     consumer.wakeup();
- * </pre>
- *
- * <p>
- * We have intentionally avoided implementing a particular threading model for processing. This leaves several
- * options for implementing multi-threaded processing of records.
- *
- * 
- * <h4>1. One Consumer Per Thread</h4>
- * 
- * A simple option is to give each thread it's own consumer instance. Here are the pros and cons of this approach:
- * <ul>
- * <li><b>PRO</b>: It is the easiest to implement
- * <li><b>PRO</b>: It is often the fastest as no inter-thread co-ordination is needed
- * <li><b>PRO</b>: It makes in-order processing on a per-partition basis very easy to implement (each thread just
- * processes messages in the order it receives them).
- * <li><b>CON</b>: More consumers means more TCP connections to the cluster (one per thread). In general Kafka handles
- * connections very efficiently so this is generally a small cost.
- * <li><b>CON</b>: Multiple consumers means more requests being sent to the server and slightly less batching of data
- * which can cause some drop in I/O throughput.
- * <li><b>CON</b>: The number of total threads across all processes will be limited by the total number of partitions.
- * </ul>
- * 
- * <h4>2. Decouple Consumption and Processing</h4>
- * 
- * Another alternative is to have one or more consumer threads that do all data consumption and hands off
- * {@link ConsumerRecords} instances to a blocking queue consumed by a pool of processor threads that actually handle
- * the record processing.
- * 
- * This option likewise has pros and cons:
- * <ul>
- * <li><b>PRO</b>: This option allows independently scaling the number of consumers and processors. This makes it
- * possible to have a single consumer that feeds many processor threads, avoiding any limitation on partitions.
- * <li><b>CON</b>: Guaranteeing order across the processors requires particular care as the threads will execute
- * independently an earlier chunk of data may actually be processed after a later chunk of data just due to the luck of
- * thread execution timing. For processing that has no ordering requirements this is not a problem.
- * <li><b>CON</b>: Manually committing the position becomes harder as it requires that all threads co-ordinate to ensure
- * that processing is complete for that partition.
- * </ul>
- * 
- * There are many possible variations on this approach. For example each processor thread can have it's own queue, and
- * the consumer threads can hash into these queues using the TopicPartition to ensure in-order consumption and simplify
- * commit.
- * 
- */
-public class KafkaConsumer<K, V> implements Consumer<K, V> {
-
-    private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class);
-    private static final long NO_CURRENT_THREAD = -1L;
-    private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1);
-
-    private final Coordinator coordinator;
-    private final Deserializer<K> keyDeserializer;
-    private final Deserializer<V> valueDeserializer;
-    private final Fetcher<K, V> fetcher;
-
-    private final Time time;
-    private final ConsumerNetworkClient client;
-    private final Metrics metrics;
-    private final SubscriptionState subscriptions;
-    private final Metadata metadata;
-    private final long retryBackoffMs;
-    private final boolean autoCommit;
-    private final long autoCommitIntervalMs;
-    private boolean closed = false;
-
-    // currentThread holds the threadId of the current thread accessing KafkaConsumer
-    // and is used to prevent multi-threaded access
-    private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD);
-    // refcount is used to allow reentrant access by the thread who has acquired currentThread
-    private final AtomicInteger refcount = new AtomicInteger(0);
-
-    // TODO: This timeout controls how long we should wait before retrying a request. We should be able
-    //       to leverage the work of KAFKA-2120 to get this value from configuration.
-    private long requestTimeoutMs = 5000L;
-
-    /**
-     * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
-     * are documented <a href="http://kafka.apache.org/documentation.html#consumerconfigs" >here</a>. Values can be
-     * either strings or objects of the appropriate type (for example a numeric configuration would accept either the
-     * string "42" or the integer 42).
-     * <p>
-     * Valid configuration strings are documented at {@link ConsumerConfig}
-     * 
-     * @param configs The consumer configs
-     */
-    public KafkaConsumer(Map<String, Object> configs) {
-        this(configs, null, null, null);
-    }
-
-    /**
-     * A consumer is instantiated by providing a set of key-value pairs as configuration, a
-     * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}.
-     * <p>
-     * Valid configuration strings are documented at {@link ConsumerConfig}
-     * 
-     * @param configs The consumer configs
-     * @param callback A callback interface that the user can implement to manage customized offsets on the start and
-     *            end of every rebalance operation.
-     * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method
-     *            won't be called in the consumer when the deserializer is passed in directly.
-     * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method
-     *            won't be called in the consumer when the deserializer is passed in directly.
-     */
-    public KafkaConsumer(Map<String, Object> configs,
-                         ConsumerRebalanceCallback callback,
-                         Deserializer<K> keyDeserializer,
-                         Deserializer<V> valueDeserializer) {
-        this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)),
-            callback,
-            keyDeserializer,
-            valueDeserializer);
-    }
-
-    /**
-     * A consumer is instantiated by providing a {@link Properties} object as configuration. Valid
-     * configuration strings are documented at {@link ConsumerConfig} A consumer is instantiated by providing a
-     * {@link Properties} object as configuration. Valid configuration strings are documented at
-     * {@link ConsumerConfig}
-     */
-    public KafkaConsumer(Properties properties) {
-        this(properties, null, null, null);
-    }
-
-    /**
-     * A consumer is instantiated by providing a {@link Properties} object as configuration and a
-     * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}.
-     * <p>
-     * Valid configuration strings are documented at {@link ConsumerConfig}
-     * 
-     * @param properties The consumer configuration properties
-     * @param callback A callback interface that the user can implement to manage customized offsets on the start and
-     *            end of every rebalance operation.
-     * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method
-     *            won't be called in the consumer when the deserializer is passed in directly.
-     * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method
-     *            won't be called in the consumer when the deserializer is passed in directly.
-     */
-    public KafkaConsumer(Properties properties,
-                         ConsumerRebalanceCallback callback,
-                         Deserializer<K> keyDeserializer,
-                         Deserializer<V> valueDeserializer) {
-        this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)),
-             callback,
-             keyDeserializer,
-             valueDeserializer);
-    }
-
-    @SuppressWarnings("unchecked")
-    private KafkaConsumer(ConsumerConfig config,
-                          ConsumerRebalanceCallback callback,
-                          Deserializer<K> keyDeserializer,
-                          Deserializer<V> valueDeserializer) {
-        try {
-            log.debug("Starting the Kafka consumer");
-            if (callback == null)
-                callback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
-                        ConsumerRebalanceCallback.class);
-            this.time = new SystemTime();
-            this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
-            this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
-
-            MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
-                    .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
-                            TimeUnit.MILLISECONDS);
-            String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
-            String jmxPrefix = "kafka.consumer";
-            if (clientId.length() <= 0)
-                clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement();
-            List<MetricsReporter> reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG,
-                    MetricsReporter.class);
-            reporters.add(new JmxReporter(jmxPrefix));
-            this.metrics = new Metrics(metricConfig, reporters, time);
-            this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
-            this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG));
-            List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
-            this.metadata.update(Cluster.bootstrap(addresses), 0);
-
-            String metricGrpPrefix = "consumer";
-            Map<String, String> metricsTags = new LinkedHashMap<String, String>();
-            metricsTags.put("client-id", clientId);
-            NetworkClient netClient = new NetworkClient(
-                    new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags),
-                    this.metadata,
-                    clientId,
-                    100, // a fixed large enough value will suffice
-                    config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG),
-                    config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG),
-                    config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG));
-            this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs);
-            OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase());
-            this.subscriptions = new SubscriptionState(offsetResetStrategy);
-            this.coordinator = new Coordinator(this.client,
-                    config.getString(ConsumerConfig.GROUP_ID_CONFIG),
-                    config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG),
-                    config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
-                    this.subscriptions,
-                    metrics,
-                    metricGrpPrefix,
-                    metricsTags,
-                    this.time,
-                    requestTimeoutMs,
-                    retryBackoffMs,
-                    wrapRebalanceCallback(callback));
-            if (keyDeserializer == null) {
-                this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
-                        Deserializer.class);
-                this.keyDeserializer.configure(config.originals(), true);
-            } else {
-                this.keyDeserializer = keyDeserializer;
-            }
-            if (valueDeserializer == null) {
-                this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
-                        Deserializer.class);
-                this.valueDeserializer.configure(config.originals(), false);
-            } else {
-                this.valueDeserializer = valueDeserializer;
-            }
-            this.fetcher = new Fetcher<K, V>(this.client,
-                    config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG),
-                    config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG),
-                    config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG),
-                    config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG),
-                    this.keyDeserializer,
-                    this.valueDeserializer,
-                    this.metadata,
-                    this.subscriptions,
-                    metrics,
-                    metricGrpPrefix,
-                    metricsTags,
-                    this.time,
-                    this.retryBackoffMs);
-
-            config.logUnused();
-
-            if (autoCommit)
-                scheduleAutoCommitTask(autoCommitIntervalMs);
-
-            log.debug("Kafka consumer created");
-        } catch (Throwable t) {
-            // call close methods if internal objects are already constructed
-            // this is to prevent resource leak. see KAFKA-2121
-            close(true);
-            // now propagate the exception
-            throw new KafkaException("Failed to construct kafka consumer", t);
-        }
-    }
-
-    /**
-     * The set of partitions currently assigned to this consumer. If subscription happened by directly subscribing to
-     * partitions using {@link #subscribe(TopicPartition...)} then this will simply return the list of partitions that
-     * were subscribed to. If subscription was done by specifying only the topic using {@link #subscribe(String...)}
-     * then this will give the set of topics currently assigned to the consumer (which may be none if the assignment
-     * hasn't happened yet, or the partitions are in the process of getting reassigned).
-     */
-    public Set<TopicPartition> subscriptions() {
-        acquire();
-        try {
-            return Collections.unmodifiableSet(this.subscriptions.assignedPartitions());
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Incrementally subscribes to the given list of topics and uses the consumer's group management functionality
-     * <p>
-     * As part of group management, the consumer will keep track of the list of consumers that belong to a particular
-     * group and will trigger a rebalance operation if one of the following events trigger -
-     * <ul>
-     * <li>Number of partitions change for any of the subscribed list of topics
-     * <li>Topic is created or deleted
-     * <li>An existing member of the consumer group dies
-     * <li>A new member is added to an existing consumer group via the join API
-     * </ul>
-     * 
-     * @param topics A variable list of topics that the consumer wants to subscribe to
-     */
-    @Override
-    public void subscribe(String... topics) {
-        acquire();
-        try {
-            log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", "));
-            for (String topic : topics)
-                this.subscriptions.subscribe(topic);
-            metadata.addTopics(topics);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Incrementally subscribes to a specific topic partition and does not use the consumer's group management
-     * functionality. As such, there will be no rebalance operation triggered when group membership or cluster and topic
-     * metadata change.
-     * <p>
-     *
-     * @param partitions Partitions to incrementally subscribe to
-     */
-    @Override
-    public void subscribe(TopicPartition... partitions) {
-        acquire();
-        try {
-            log.debug("Subscribed to partitions(s): {}", Utils.join(partitions, ", "));
-            for (TopicPartition tp : partitions) {
-                this.subscriptions.subscribe(tp);
-                metadata.addTopics(tp.topic());
-            }
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Unsubscribe from the specific topics. This will trigger a rebalance operation and records for this topic will not
-     * be returned from the next {@link #poll(long) poll()} onwards
-     * 
-     * @param topics Topics to unsubscribe from
-     */
-    public void unsubscribe(String... topics) {
-        acquire();
-        try {
-            log.debug("Unsubscribed from topic(s): {}", Utils.join(topics, ", "));
-            // throw an exception if the topic was never subscribed to
-            for (String topic : topics)
-                this.subscriptions.unsubscribe(topic);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Unsubscribe from the specific topic partitions. records for these partitions will not be returned from the next
-     * {@link #poll(long) poll()} onwards
-     * 
-     * @param partitions Partitions to unsubscribe from
-     */
-    public void unsubscribe(TopicPartition... partitions) {
-        acquire();
-        try {
-            log.debug("Unsubscribed from partitions(s): {}", Utils.join(partitions, ", "));
-            // throw an exception if the partition was never subscribed to
-            for (TopicPartition partition : partitions)
-                this.subscriptions.unsubscribe(partition);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Fetches data for the topics or partitions specified using one of the subscribe APIs. It is an error to not have
-     * subscribed to any topics or partitions before polling for data.
-     * <p>
-     * The offset used for fetching the data is governed by whether or not {@link #seek(TopicPartition, long)} is used.
-     * If {@link #seek(TopicPartition, long)} is used, it will use the specified offsets on startup and on every
-     * rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed
-     * offset using {@link #commit(Map, CommitType) commit(offsets, sync)} for the subscribed list of partitions.
-     * 
-     * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, returns
-     *            immediately with any records available now. Must not be negative.
-     * @return map of topic to records since the last fetch for the subscribed list of topics and partitions
-     * 
-     * @throws NoOffsetForPartitionException If there is no stored offset for a subscribed partition and no automatic
-     *             offset reset policy has been configured.
-     */
-    @Override
-    public ConsumerRecords<K, V> poll(long timeout) {
-        acquire();
-        try {
-            if (timeout < 0)
-                throw new IllegalArgumentException("Timeout must not be negative");
-
-            // poll for new data until the timeout expires
-            long remaining = timeout;
-            while (remaining >= 0) {
-                long start = time.milliseconds();
-                Map<TopicPartition, List<ConsumerRecord<K, V>>> records = pollOnce(remaining);
-                long end = time.milliseconds();
-
-                if (!records.isEmpty()) {
-                    // if data is available, then return it, but first send off the
-                    // next round of fetches to enable pipelining while the user is
-                    // handling the fetched records.
-                    fetcher.initFetches(metadata.fetch());
-                    client.poll(0);
-                    return new ConsumerRecords<K, V>(records);
-                }
-
-                remaining -= end - start;
-
-                // nothing was available, so we should backoff before retrying
-                if (remaining > 0) {
-                    Utils.sleep(min(remaining, retryBackoffMs));
-                    remaining -= time.milliseconds() - end;
-                }
-            }
-
-            return ConsumerRecords.empty();
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Do one round of polling. In addition to checking for new data, this does any needed
-     * heart-beating, auto-commits, and offset updates.
-     * @param timeout The maximum time to block in the underlying poll
-     * @return The fetched records (may be empty)
-     */
-    private Map<TopicPartition, List<ConsumerRecord<K, V>>> pollOnce(long timeout) {
-        // TODO: Sub-requests should take into account the poll timeout (KAFKA-1894)
-        coordinator.ensureCoordinatorKnown();
-
-        // ensure we have partitions assigned if we expect to
-        if (subscriptions.partitionsAutoAssigned())
-            coordinator.ensurePartitionAssignment();
-
-        // fetch positions if we have partitions we're subscribed to that we
-        // don't know the offset for
-        if (!subscriptions.hasAllFetchPositions())
-            updateFetchPositions(this.subscriptions.missingFetchPositions());
-
-        // init any new fetches (won't resend pending fetches)
-        Cluster cluster = this.metadata.fetch();
-        fetcher.initFetches(cluster);
-        client.poll(timeout);
-        return fetcher.fetchedRecords();
-    }
-
-    private void scheduleAutoCommitTask(final long interval) {
-        DelayedTask task = new DelayedTask() {
-            public void run(long now) {
-                commit(CommitType.ASYNC);
-                client.schedule(this, now + interval);
-            }
-        };
-        client.schedule(task, time.milliseconds() + interval);
-    }
-
-    /**
-     * Commits the specified offsets for the specified list of topics and partitions to Kafka.
-     * <p>
-     * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every
-     * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
-     * should not be used.
-     * <p>
-     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
-     * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use
-     * {@link #commit(Map, CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC})
-     * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown
-     * to the caller).
-     *
-     * @param offsets The list of offsets per partition that should be committed to Kafka.
-     * @param commitType Control whether the commit is blocking
-     */
-    @Override
-    public void commit(final Map<TopicPartition, Long> offsets, CommitType commitType) {
-        commit(offsets, commitType, null);
-    }
-
-    /**
-     * Commits the specified offsets for the specified list of topics and partitions to Kafka.
-     * <p>
-     * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every
-     * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
-     * should not be used.
-     * <p>
-     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
-     * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e.
-     * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In
-     * this case, the error is either passed to the callback (if provided) or thrown to the caller.
-     *
-     * @param offsets The list of offsets per partition that should be committed to Kafka.
-     * @param commitType Control whether the commit is blocking
-     * @param callback Callback to invoke when the commit completes
-     */
-    @Override
-    public void commit(final Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
-        acquire();
-        try {
-            log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets);
-            coordinator.commitOffsets(offsets, commitType, callback);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions.
-     * <p>
-     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
-     * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
-     * should not be used.
-     * <p>
-     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
-     * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e.
-     * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In
-     * this case, the error is either passed to the callback (if provided) or thrown to the caller.
-     *
-     * @param commitType Whether or not the commit should block until it is acknowledged.
-     * @param callback Callback to invoke when the commit completes
-     */
-    @Override
-    public void commit(CommitType commitType, ConsumerCommitCallback callback) {
-        acquire();
-        try {
-            // need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance)
-            Map<TopicPartition, Long> allConsumed = new HashMap<TopicPartition, Long>(this.subscriptions.allConsumed());
-            commit(allConsumed, commitType, callback);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions.
-     * <p>
-     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
-     * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
-     * should not be used.
-     * <p>
-     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
-     * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use
-     * {@link #commit(CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC})
-     * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown
-     * to the caller).
-     *
-     * @param commitType Whether or not the commit should block until it is acknowledged.
-     */
-    @Override
-    public void commit(CommitType commitType) {
-        commit(commitType, null);
-    }
-
-    /**
-     * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API
-     * is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that
-     * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets
-     */
-    @Override
-    public void seek(TopicPartition partition, long offset) {
-        acquire();
-        try {
-            log.debug("Seeking to offset {} for partition {}", offset, partition);
-            this.subscriptions.seek(partition, offset);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Seek to the first offset for each of the given partitions
-     */
-    public void seekToBeginning(TopicPartition... partitions) {
-        acquire();
-        try {
-            Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
-                    : Arrays.asList(partitions);
-            for (TopicPartition tp : parts) {
-                log.debug("Seeking to beginning of partition {}", tp);
-                subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST);
-            }
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Seek to the last offset for each of the given partitions
-     */
-    public void seekToEnd(TopicPartition... partitions) {
-        acquire();
-        try {
-            Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
-                    : Arrays.asList(partitions);
-            for (TopicPartition tp : parts) {
-                log.debug("Seeking to end of partition {}", tp);
-                subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST);
-            }
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Returns the offset of the <i>next record</i> that will be fetched (if a record with that offset exists).
-     * 
-     * @param partition The partition to get the position for
-     * @return The offset
-     * @throws NoOffsetForPartitionException If a position hasn't been set for a given partition, and no reset policy is
-     *             available.
-     */
-    public long position(TopicPartition partition) {
-        acquire();
-        try {
-            if (!this.subscriptions.assignedPartitions().contains(partition))
-                throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer.");
-            Long offset = this.subscriptions.consumed(partition);
-            if (offset == null) {
-                updateFetchPositions(Collections.singleton(partition));
-                return this.subscriptions.consumed(partition);
-            } else {
-                return offset;
-            }
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Fetches the last committed offset for the given partition (whether the commit happened by this process or
-     * another). This offset will be used as the position for the consumer in the event of a failure.
-     * <p>
-     * This call may block to do a remote call if the partition in question isn't assigned to this consumer or if the
-     * consumer hasn't yet initialized it's cache of committed offsets.
-     * 
-     * @param partition The partition to check
-     * @return The last committed offset or null if no offset has been committed
-     * @throws NoOffsetForPartitionException If no offset has ever been committed by any process for the given
-     *             partition.
-     */
-    @Override
-    public long committed(TopicPartition partition) {
-        acquire();
-        try {
-            Long committed;
-            if (subscriptions.assignedPartitions().contains(partition)) {
-                committed = this.subscriptions.committed(partition);
-                if (committed == null) {
-                    coordinator.refreshCommittedOffsetsIfNeeded();
-                    committed = this.subscriptions.committed(partition);
-                }
-            } else {
-                Map<TopicPartition, Long> offsets = coordinator.fetchCommittedOffsets(Collections.singleton(partition));
-                committed = offsets.get(partition);
-            }
-
-            if (committed == null)
-                throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition);
-
-            return committed;
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Get the metrics kept by the consumer
-     */
-    @Override
-    public Map<MetricName, ? extends Metric> metrics() {
-        return Collections.unmodifiableMap(this.metrics.metrics());
-    }
-
-    /**
-     * Get metadata about the partitions for a given topic. This method will issue a remote call to the server if it
-     * does not already have any metadata about the given topic.
-     * 
-     * @param topic The topic to get partition metadata for
-     * @return The list of partitions
-     */
-    @Override
-    public List<PartitionInfo> partitionsFor(String topic) {
-        acquire();
-        try {
-            Cluster cluster = this.metadata.fetch();
-            List<PartitionInfo> parts = cluster.partitionsForTopic(topic);
-            if (parts == null) {
-                metadata.add(topic);
-                client.awaitMetadataUpdate();
-                parts = metadata.fetch().partitionsForTopic(topic);
-            }
-            return parts;
-        } finally {
-            release();
-        }
-    }
-
-    @Override
-    public void close() {
-        acquire();
-        try {
-            if (closed) return;
-            close(false);
-        } finally {
-            release();
-        }
-    }
-
-    /**
-     * Wakeup the consumer. This method is thread-safe and is useful in particular to abort a long poll.
-     * The thread which is blocking in an operation will throw {@link ConsumerWakeupException}.
-     */
-    @Override
-    public void wakeup() {
-        this.client.wakeup();
-    }
-
-    private void close(boolean swallowException) {
-        log.trace("Closing the Kafka consumer.");
-        AtomicReference<Throwable> firstException = new AtomicReference<Throwable>();
-        this.closed = true;
-        ClientUtils.closeQuietly(metrics, "consumer metrics", firstException);
-        ClientUtils.closeQuietly(client, "consumer network client", firstException);
-        ClientUtils.closeQuietly(keyDeserializer, "consumer key deserializer", firstException);
-        ClientUtils.closeQuietly(valueDeserializer, "consumer value deserializer", firstException);
-        log.debug("The Kafka consumer has closed.");
-        if (firstException.get() != null && !swallowException) {
-            throw new KafkaException("Failed to close kafka consumer", firstException.get());
-        }
-    }
-
-    private Coordinator.RebalanceCallback wrapRebalanceCallback(final ConsumerRebalanceCallback callback) {
-        return new Coordinator.RebalanceCallback() {
-            @Override
-            public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
-                callback.onPartitionsAssigned(KafkaConsumer.this, partitions);
-            }
-
-            @Override
-            public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
-                callback.onPartitionsRevoked(KafkaConsumer.this, partitions);
-            }
-        };
-    }
-
-    /**
-     * Set the fetch position to the committed position (if there is one)
-     * or reset it using the offset reset policy the user has configured.
-     *
-     * @param partitions The partitions that needs updating fetch positions
-     * @throws NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is
-     *             defined
-     */
-    private void updateFetchPositions(Set<TopicPartition> partitions) {
-        // refresh commits for all assigned partitions
-        coordinator.refreshCommittedOffsetsIfNeeded();
-
-        // then do any offset lookups in case some positions are not known
-        fetcher.updateFetchPositions(partitions);
-    }
-
-    /*
-     * Check that the consumer hasn't been closed.
-     */
-    private void ensureNotClosed() {
-        if (this.closed)
-            throw new IllegalStateException("This consumer has already been closed.");
-    }
-
-    /**
-     * Acquire the light lock protecting this consumer from multi-threaded access. Instead of blocking
-     * when the lock is not available, however, we just throw an exception (since multi-threaded usage is not
-     * supported).
-     * @throws IllegalStateException if the consumer has been closed
-     * @throws ConcurrentModificationException if another thread already has the lock
-     */
-    private void acquire() {
-        ensureNotClosed();
-        long threadId = Thread.currentThread().getId();
-        if (threadId != currentThread.get() && !currentThread.compareAndSet(NO_CURRENT_THREAD, threadId))
-            throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access");
-        refcount.incrementAndGet();
-    }
-
-    /**
-     * Release the light lock protecting the consumer from multi-threaded access.
-     */
-    private void release() {
-        if (refcount.decrementAndGet() == 0)
-            currentThread.set(NO_CURRENT_THREAD);
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/MockConsumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/MockConsumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/MockConsumer.java
deleted file mode 100644
index 1d08519..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/MockConsumer.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer;
-
-import org.apache.flink.kafka_backport.clients.consumer.internals.SubscriptionState;
-import org.apache.flink.kafka_backport.common.Metric;
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.apache.flink.kafka_backport.common.PartitionInfo;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. This class is <i> not
- * threadsafe </i>
- * <p>
- * The consumer runs in the user thread and multiplexes I/O over TCP connections to each of the brokers it needs to
- * communicate with. Failure to close the consumer after use will leak these resources.
- */
-public class MockConsumer<K, V> implements Consumer<K, V> {
-
-    private final Map<String, List<PartitionInfo>> partitions;
-    private final SubscriptionState subscriptions;
-    private Map<TopicPartition, List<ConsumerRecord<K, V>>> records;
-    private boolean closed;
-
-    public MockConsumer(OffsetResetStrategy offsetResetStrategy) {
-        this.subscriptions = new SubscriptionState(offsetResetStrategy);
-        this.partitions = new HashMap<String, List<PartitionInfo>>();
-        this.records = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
-        this.closed = false;
-    }
-    
-    @Override
-    public synchronized Set<TopicPartition> subscriptions() {
-        return this.subscriptions.assignedPartitions();
-    }
-
-    @Override
-    public synchronized void subscribe(String... topics) {
-        ensureNotClosed();
-        for (String topic : topics)
-            this.subscriptions.subscribe(topic);
-    }
-
-    @Override
-    public synchronized void subscribe(TopicPartition... partitions) {
-        ensureNotClosed();
-        for (TopicPartition partition : partitions)
-            this.subscriptions.subscribe(partition);
-    }
-
-    public synchronized void unsubscribe(String... topics) {
-        ensureNotClosed();
-        for (String topic : topics)
-            this.subscriptions.unsubscribe(topic);
-    }
-
-    public synchronized void unsubscribe(TopicPartition... partitions) {
-        ensureNotClosed();
-        for (TopicPartition partition : partitions)
-            this.subscriptions.unsubscribe(partition);
-    }
-
-    @Override
-    public synchronized ConsumerRecords<K, V> poll(long timeout) {
-        ensureNotClosed();
-        // update the consumed offset
-        for (Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : this.records.entrySet()) {
-            List<ConsumerRecord<K, V>> recs = entry.getValue();
-            if (!recs.isEmpty())
-                this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset());
-        }
-
-        ConsumerRecords<K, V> copy = new ConsumerRecords<K, V>(this.records);
-        this.records = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
-        return copy;
-    }
-
-    public synchronized void addRecord(ConsumerRecord<K, V> record) {
-        ensureNotClosed();
-        TopicPartition tp = new TopicPartition(record.topic(), record.partition());
-        this.subscriptions.assignedPartitions().add(tp);
-        List<ConsumerRecord<K, V>> recs = this.records.get(tp);
-        if (recs == null) {
-            recs = new ArrayList<ConsumerRecord<K, V>>();
-            this.records.put(tp, recs);
-        }
-        recs.add(record);
-    }
-
-    @Override
-    public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
-        ensureNotClosed();
-        for (Entry<TopicPartition, Long> entry : offsets.entrySet())
-            subscriptions.committed(entry.getKey(), entry.getValue());
-        if (callback != null) {
-            callback.onComplete(offsets, null);
-        }
-    }
-
-    @Override
-    public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType) {
-        commit(offsets, commitType, null);
-    }
-
-    @Override
-    public synchronized void commit(CommitType commitType, ConsumerCommitCallback callback) {
-        ensureNotClosed();
-        commit(this.subscriptions.allConsumed(), commitType, callback);
-    }
-
-    @Override
-    public synchronized void commit(CommitType commitType) {
-        commit(commitType, null);
-    }
-
-    @Override
-    public synchronized void seek(TopicPartition partition, long offset) {
-        ensureNotClosed();
-        subscriptions.seek(partition, offset);
-    }
-
-    @Override
-    public synchronized long committed(TopicPartition partition) {
-        ensureNotClosed();
-        return subscriptions.committed(partition);
-    }
-
-    @Override
-    public synchronized long position(TopicPartition partition) {
-        ensureNotClosed();
-        return subscriptions.consumed(partition);
-    }
-
-    @Override
-    public synchronized void seekToBeginning(TopicPartition... partitions) {
-        ensureNotClosed();
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public synchronized void seekToEnd(TopicPartition... partitions) {
-        ensureNotClosed();
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public Map<MetricName, ? extends Metric> metrics() {
-        ensureNotClosed();
-        return Collections.emptyMap();
-    }
-
-    @Override
-    public synchronized List<PartitionInfo> partitionsFor(String topic) {
-        ensureNotClosed();
-        List<PartitionInfo> parts = this.partitions.get(topic);
-        if (parts == null)
-            return Collections.emptyList();
-        else
-            return parts;
-    }
-
-    public synchronized void updatePartitions(String topic, List<PartitionInfo> partitions) {
-        ensureNotClosed();
-        this.partitions.put(topic, partitions);
-    }
-
-    @Override
-    public synchronized void close() {
-        ensureNotClosed();
-        this.closed = true;
-    }
-
-    @Override
-    public void wakeup() {
-
-    }
-
-    private void ensureNotClosed() {
-        if (this.closed)
-            throw new IllegalStateException("This consumer has already been closed.");
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/NoOffsetForPartitionException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/NoOffsetForPartitionException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/NoOffsetForPartitionException.java
deleted file mode 100644
index 19ae0a6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/NoOffsetForPartitionException.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.kafka_backport.clients.consumer;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Indicates that there is no stored offset and no defined offset reset policy
- */
-public class NoOffsetForPartitionException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public NoOffsetForPartitionException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/OffsetResetStrategy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/OffsetResetStrategy.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/OffsetResetStrategy.java
deleted file mode 100644
index 70c254f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/OffsetResetStrategy.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public enum OffsetResetStrategy {
-    LATEST, EARLIEST, NONE
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/ConsumerNetworkClient.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/ConsumerNetworkClient.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/ConsumerNetworkClient.java
deleted file mode 100644
index a6d16cd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/ConsumerNetworkClient.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients.consumer.internals;
-
-import org.apache.flink.kafka_backport.clients.ClientResponse;
-import org.apache.flink.kafka_backport.clients.Metadata;
-import org.apache.flink.kafka_backport.clients.consumer.ConsumerWakeupException;
-import org.apache.flink.kafka_backport.common.Node;
-import org.apache.flink.kafka_backport.common.requests.AbstractRequest;
-import org.apache.flink.kafka_backport.common.requests.RequestHeader;
-import org.apache.flink.kafka_backport.common.requests.RequestSend;
-import org.apache.flink.kafka_backport.clients.ClientRequest;
-import org.apache.flink.kafka_backport.clients.KafkaClient;
-import org.apache.flink.kafka_backport.clients.RequestCompletionHandler;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.utils.Time;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-/**
- * Higher level consumer access to the network layer with basic support for futures and
- * task scheduling. NOT thread-safe!
- *
- * TODO: The current implementation is simplistic in that it provides a facility for queueing requests
- * prior to delivery, but it makes no effort to retry requests which cannot be sent at the time
- * {@link #poll(long)} is called. This makes the behavior of the queue predictable and easy to
- * understand, but there are opportunities to provide timeout or retry capabilities in the future.
- * How we do this may depend on KAFKA-2120, so for now, we retain the simplistic behavior.
- */
-public class ConsumerNetworkClient implements Closeable {
-    private final KafkaClient client;
-    private final AtomicBoolean wakeup = new AtomicBoolean(false);
-    private final DelayedTaskQueue delayedTasks = new DelayedTaskQueue();
-    private final Map<Node, List<ClientRequest>> unsent = new HashMap<Node, List<ClientRequest>>();
-    private final Metadata metadata;
-    private final Time time;
-    private final long retryBackoffMs;
-
-    public ConsumerNetworkClient(KafkaClient client,
-                                 Metadata metadata,
-                                 Time time,
-                                 long retryBackoffMs) {
-        this.client = client;
-        this.metadata = metadata;
-        this.time = time;
-        this.retryBackoffMs = retryBackoffMs;
-    }
-
-    /**
-     * Schedule a new task to be executed at the given time. This is "best-effort" scheduling and
-     * should only be used for coarse synchronization.
-     * @param task The task to be scheduled
-     * @param at The time it should run
-     */
-    public void schedule(DelayedTask task, long at) {
-        delayedTasks.add(task, at);
-    }
-
-    /**
-     * Unschedule a task. This will remove all instances of the task from the task queue.
-     * This is a no-op if the task is not scheduled.
-     * @param task The task to be unscheduled.
-     */
-    public void unschedule(DelayedTask task) {
-        delayedTasks.remove(task);
-    }
-
-    /**
-     * Send a new request. Note that the request is not actually transmitted on the
-     * network until one of the {@link #poll(long)} variants is invoked. At this
-     * point the request will either be transmitted successfully or will fail.
-     * Use the returned future to obtain the result of the send.
-     * @param node The destination of the request
-     * @param api The Kafka API call
-     * @param request The request payload
-     * @return A future which indicates the result of the send.
-     */
-    public RequestFuture<ClientResponse> send(Node node,
-                                              ApiKeys api,
-                                              AbstractRequest request) {
-        long now = time.milliseconds();
-        RequestFutureCompletionHandler future = new RequestFutureCompletionHandler();
-        RequestHeader header = client.nextRequestHeader(api);
-        RequestSend send = new RequestSend(node.idString(), header, request.toStruct());
-        put(node, new ClientRequest(now, true, send, future));
-        return future;
-    }
-
-    private void put(Node node, ClientRequest request) {
-        List<ClientRequest> nodeUnsent = unsent.get(node);
-        if (nodeUnsent == null) {
-            nodeUnsent = new ArrayList<ClientRequest>();
-            unsent.put(node, nodeUnsent);
-        }
-        nodeUnsent.add(request);
-    }
-
-    public Node leastLoadedNode() {
-        return client.leastLoadedNode(time.milliseconds());
-    }
-
-    /**
-     * Block until the metadata has been refreshed.
-     */
-    public void awaitMetadataUpdate() {
-        int version = this.metadata.requestUpdate();
-        do {
-            poll(Long.MAX_VALUE);
-        } while (this.metadata.version() == version);
-    }
-
-    /**
-     * Wakeup an active poll. This will cause the polling thread to throw an exception either
-     * on the current poll if one is active, or the next poll.
-     */
-    public void wakeup() {
-        this.wakeup.set(true);
-        this.client.wakeup();
-    }
-
-    /**
-     * Block indefinitely until the given request future has finished.
-     * @param future The request future to await.
-     * @throws org.apache.flink.kafka_backport.clients.consumer.ConsumerWakeupException if {@link #wakeup()} is called from another thread
-     */
-    public void poll(RequestFuture<?> future) {
-        while (!future.isDone())
-            poll(Long.MAX_VALUE);
-    }
-
-    /**
-     * Block until the provided request future request has finished or the timeout has expired.
-     * @param future The request future to wait for
-     * @param timeout The maximum duration (in ms) to wait for the request
-     * @return true if the future is done, false otherwise
-     * @throws org.apache.flink.kafka_backport.clients.consumer.ConsumerWakeupException if {@link #wakeup()} is called from another thread
-     */
-    public boolean poll(RequestFuture<?> future, long timeout) {
-        long now = time.milliseconds();
-        long deadline = now + timeout;
-        while (!future.isDone() && now < deadline) {
-            poll(deadline - now, now);
-            now = time.milliseconds();
-        }
-        return future.isDone();
-    }
-
-    /**
-     * Poll for any network IO. All send requests will either be transmitted on the network
-     * or failed when this call completes.
-     * @param timeout The maximum time to wait for an IO event.
-     * @throws org.apache.flink.kafka_backport.clients.consumer.ConsumerWakeupException if {@link #wakeup()} is called from another thread
-     */
-    public void poll(long timeout) {
-        poll(timeout, time.milliseconds());
-    }
-
-    private void poll(long timeout, long now) {
-        // send all the requests we can send now
-        pollUnsentRequests(now);
-
-        // ensure we don't poll any longer than the deadline for
-        // the next scheduled task
-        timeout = Math.min(timeout, delayedTasks.nextTimeout(now));
-        clientPoll(timeout, now);
-
-        // execute scheduled tasks
-        now = time.milliseconds();
-        delayedTasks.poll(now);
-
-        // try again to send requests since buffer space may have been
-        // cleared or a connect finished in the poll
-        pollUnsentRequests(now);
-
-        // fail all requests that couldn't be sent
-        clearUnsentRequests(now);
-
-    }
-
-    /**
-     * Block until all pending requests from the given node have finished.
-     * @param node The node to await requests from
-     */
-    public void awaitPendingRequests(Node node) {
-        while (pendingRequestCount(node) > 0)
-            poll(retryBackoffMs);
-    }
-
-    /**
-     * Get the count of pending requests to the given node. This includes both request that
-     * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission.
-     * @param node The node in question
-     * @return The number of pending requests
-     */
-    public int pendingRequestCount(Node node) {
-        List<ClientRequest> pending = unsent.get(node);
-        int unsentCount = pending == null ? 0 : pending.size();
-        return unsentCount + client.inFlightRequestCount(node.idString());
-    }
-
-    /**
-     * Get the total count of pending requests from all nodes. This includes both requests that
-     * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission.
-     * @return The total count of pending requests
-     */
-    public int pendingRequestCount() {
-        int total = 0;
-        for (List<ClientRequest> requests: unsent.values())
-            total += requests.size();
-        return total + client.inFlightRequestCount();
-    }
-
-    private void pollUnsentRequests(long now) {
-        while (trySend(now))
-            clientPoll(0, now);
-    }
-
-    private void clearUnsentRequests(long now) {
-        // clear all unsent requests and fail their corresponding futures
-        for (Map.Entry<Node, List<ClientRequest>> requestEntry: unsent.entrySet()) {
-            Iterator<ClientRequest> iterator = requestEntry.getValue().iterator();
-            while (iterator.hasNext()) {
-                ClientRequest request = iterator.next();
-                RequestFutureCompletionHandler handler =
-                        (RequestFutureCompletionHandler) request.callback();
-                handler.raise(SendFailedException.INSTANCE);
-                iterator.remove();
-            }
-        }
-        unsent.clear();
-    }
-
-    private boolean trySend(long now) {
-        // send any requests that can be sent now
-        boolean requestsSent = false;
-        for (Map.Entry<Node, List<ClientRequest>> requestEntry: unsent.entrySet()) {
-            Node node = requestEntry.getKey();
-            Iterator<ClientRequest> iterator = requestEntry.getValue().iterator();
-            while (iterator.hasNext()) {
-                ClientRequest request = iterator.next();
-                if (client.ready(node, now)) {
-                    client.send(request);
-                    iterator.remove();
-                    requestsSent = true;
-                } else if (client.connectionFailed(node)) {
-                    RequestFutureCompletionHandler handler =
-                            (RequestFutureCompletionHandler) request.callback();
-                    handler.onComplete(new ClientResponse(request, now, true, null));
-                    iterator.remove();
-                }
-            }
-        }
-        return requestsSent;
-    }
-
-    private void clientPoll(long timeout, long now) {
-        client.poll(timeout, now);
-        if (wakeup.get()) {
-            clearUnsentRequests(now);
-            wakeup.set(false);
-            throw new ConsumerWakeupException();
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        client.close();
-    }
-
-    public static class RequestFutureCompletionHandler
-            extends RequestFuture<ClientResponse>
-            implements RequestCompletionHandler {
-
-        @Override
-        public void onComplete(ClientResponse response) {
-            complete(response);
-        }
-    }
-}


[44/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MeasurableStat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MeasurableStat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MeasurableStat.java
deleted file mode 100644
index a3ee306..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MeasurableStat.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.kafka_backport.common.metrics;
-
-import org.apache.flink.kafka_backport.common.metrics.stats.Count;
-import org.apache.flink.kafka_backport.common.metrics.stats.Max;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A MeasurableStat is a {@link Stat} that is also {@link Measurable} (i.e. can produce a single floating point value).
- * This is the interface used for most of the simple statistics such as {@link org.apache.flink.kafka_backport.common.metrics.stats.Avg},
- * {@link Max}, {@link Count}, etc.
- */
-public interface MeasurableStat extends Stat, Measurable {
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricConfig.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricConfig.java
deleted file mode 100644
index f95824a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricConfig.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.kafka_backport.common.metrics;
-
-import java.util.concurrent.TimeUnit;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Configuration values for metrics
- */
-public class MetricConfig {
-
-    private Quota quota;
-    private int samples;
-    private long eventWindow;
-    private long timeWindowMs;
-    private TimeUnit unit;
-
-    public MetricConfig() {
-        super();
-        this.quota = null;
-        this.samples = 2;
-        this.eventWindow = Long.MAX_VALUE;
-        this.timeWindowMs = TimeUnit.MILLISECONDS.convert(30, TimeUnit.SECONDS);
-        this.unit = TimeUnit.SECONDS;
-    }
-
-    public Quota quota() {
-        return this.quota;
-    }
-
-    public MetricConfig quota(Quota quota) {
-        this.quota = quota;
-        return this;
-    }
-
-    public long eventWindow() {
-        return eventWindow;
-    }
-
-    public MetricConfig eventWindow(long window) {
-        this.eventWindow = window;
-        return this;
-    }
-
-    public long timeWindowMs() {
-        return timeWindowMs;
-    }
-
-    public MetricConfig timeWindow(long window, TimeUnit unit) {
-        this.timeWindowMs = TimeUnit.MILLISECONDS.convert(window, unit);
-        return this;
-    }
-
-    public int samples() {
-        return this.samples;
-    }
-
-    public MetricConfig samples(int samples) {
-        if (samples < 1)
-            throw new IllegalArgumentException("The number of samples must be at least 1.");
-        this.samples = samples;
-        return this;
-    }
-
-    public TimeUnit timeUnit() {
-        return unit;
-    }
-
-    public MetricConfig timeUnit(TimeUnit unit) {
-        this.unit = unit;
-        return this;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Metrics.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Metrics.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Metrics.java
deleted file mode 100644
index ea7e5f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Metrics.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.metrics;
-
-import org.apache.flink.kafka_backport.common.utils.SystemTime;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.apache.flink.kafka_backport.common.utils.CopyOnWriteMap;
-import org.apache.flink.kafka_backport.common.utils.Time;
-
-import java.io.Closeable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A registry of sensors and metrics.
- * <p>
- * A metric is a named, numerical measurement. A sensor is a handle to record numerical measurements as they occur. Each
- * Sensor has zero or more associated metrics. For example a Sensor might represent message sizes and we might associate
- * with this sensor a metric for the average, maximum, or other statistics computed off the sequence of message sizes
- * that are recorded by the sensor.
- * <p>
- * Usage looks something like this:
- * 
- * <pre>
- * // set up metrics:
- * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
- * Sensor sensor = metrics.sensor(&quot;message-sizes&quot;);
- * MetricName metricName = new MetricName(&quot;message-size-avg&quot;, &quot;producer-metrics&quot;);
- * sensor.add(metricName, new Avg());
- * metricName = new MetricName(&quot;message-size-max&quot;, &quot;producer-metrics&quot;);
- * sensor.add(metricName, new Max());
- * 
- * // as messages are sent we record the sizes
- * sensor.record(messageSize);
- * </pre>
- */
-public class Metrics implements Closeable {
-
-    private final MetricConfig config;
-    private final ConcurrentMap<MetricName, KafkaMetric> metrics;
-    private final ConcurrentMap<String, Sensor> sensors;
-    private final List<MetricsReporter> reporters;
-    private final Time time;
-
-    /**
-     * Create a metrics repository with no metric reporters and default configuration.
-     */
-    public Metrics() {
-        this(new MetricConfig());
-    }
-
-    /**
-     * Create a metrics repository with no metric reporters and default configuration.
-     */
-    public Metrics(Time time) {
-        this(new MetricConfig(), new ArrayList<MetricsReporter>(0), time);
-    }
-
-    /**
-     * Create a metrics repository with no reporters and the given default config. This config will be used for any
-     * metric that doesn't override its own config.
-     * @param defaultConfig The default config to use for all metrics that don't override their config
-     */
-    public Metrics(MetricConfig defaultConfig) {
-        this(defaultConfig, new ArrayList<MetricsReporter>(0), new SystemTime());
-    }
-
-    /**
-     * Create a metrics repository with a default config and the given metric reporters
-     * @param defaultConfig The default config
-     * @param reporters The metrics reporters
-     * @param time The time instance to use with the metrics
-     */
-    public Metrics(MetricConfig defaultConfig, List<MetricsReporter> reporters, Time time) {
-        this.config = defaultConfig;
-        this.sensors = new CopyOnWriteMap<String, Sensor>();
-        this.metrics = new CopyOnWriteMap<MetricName, KafkaMetric>();
-        this.reporters = Utils.notNull(reporters);
-        this.time = time;
-        for (MetricsReporter reporter : reporters)
-            reporter.init(new ArrayList<KafkaMetric>());
-    }
-
-    /**
-     * Get the sensor with the given name if it exists
-     * @param name The name of the sensor
-     * @return Return the sensor or null if no such sensor exists
-     */
-    public Sensor getSensor(String name) {
-        return this.sensors.get(Utils.notNull(name));
-    }
-
-    /**
-     * Get or create a sensor with the given unique name and no parent sensors.
-     * @param name The sensor name
-     * @return The sensor
-     */
-    public Sensor sensor(String name) {
-        return sensor(name, null, (Sensor[]) null);
-    }
-
-    /**
-     * Get or create a sensor with the given unique name and zero or more parent sensors. All parent sensors will
-     * receive every value recorded with this sensor.
-     * @param name The name of the sensor
-     * @param parents The parent sensors
-     * @return The sensor that is created
-     */
-    public Sensor sensor(String name, Sensor... parents) {
-        return sensor(name, null, parents);
-    }
-
-    /**
-     * Get or create a sensor with the given unique name and zero or more parent sensors. All parent sensors will
-     * receive every value recorded with this sensor.
-     * @param name The name of the sensor
-     * @param config A default configuration to use for this sensor for metrics that don't have their own config
-     * @param parents The parent sensors
-     * @return The sensor that is created
-     */
-    public synchronized Sensor sensor(String name, MetricConfig config, Sensor... parents) {
-        Sensor s = getSensor(name);
-        if (s == null) {
-            s = new Sensor(this, name, parents, config == null ? this.config : config, time);
-            this.sensors.put(name, s);
-        }
-        return s;
-    }
-
-    /**
-     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
-     * This is a way to expose existing values as metrics.
-     * @param metricName The name of the metric
-     * @param measurable The measurable that will be measured by this metric
-     */
-    public void addMetric(MetricName metricName, Measurable measurable) {
-        addMetric(metricName, null, measurable);
-    }
-
-    /**
-     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
-     * This is a way to expose existing values as metrics.
-     * @param metricName The name of the metric
-     * @param config The configuration to use when measuring this measurable
-     * @param measurable The measurable that will be measured by this metric
-     */
-    public synchronized void addMetric(MetricName metricName, MetricConfig config, Measurable measurable) {
-        KafkaMetric m = new KafkaMetric(new Object(),
-                                        Utils.notNull(metricName),
-                                        Utils.notNull(measurable),
-                                        config == null ? this.config : config,
-                                        time);
-        registerMetric(m);
-    }
-
-    /**
-     * Add a MetricReporter
-     */
-    public synchronized void addReporter(MetricsReporter reporter) {
-        Utils.notNull(reporter).init(new ArrayList<KafkaMetric>(metrics.values()));
-        this.reporters.add(reporter);
-    }
-
-    synchronized void registerMetric(KafkaMetric metric) {
-        MetricName metricName = metric.metricName();
-        if (this.metrics.containsKey(metricName))
-            throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one.");
-        this.metrics.put(metricName, metric);
-        for (MetricsReporter reporter : reporters)
-            reporter.metricChange(metric);
-    }
-
-    /**
-     * Get all the metrics currently maintained indexed by metricName
-     */
-    public Map<MetricName, KafkaMetric> metrics() {
-        return this.metrics;
-    }
-
-    /**
-     * Close this metrics repository.
-     */
-    @Override
-    public void close() {
-        for (MetricsReporter reporter : this.reporters)
-            reporter.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricsReporter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricsReporter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricsReporter.java
deleted file mode 100644
index 870198b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/MetricsReporter.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.kafka_backport.common.metrics;
-
-import org.apache.flink.kafka_backport.common.Configurable;
-
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A plugin interface to allow things to listen as new metrics are created so they can be reported.
- */
-public interface MetricsReporter extends Configurable {
-
-    /**
-     * This is called when the reporter is first registered to initially register all existing metrics
-     * @param metrics All currently existing metrics
-     */
-    public void init(List<KafkaMetric> metrics);
-
-    /**
-     * This is called whenever a metric is updated or added
-     * @param metric
-     */
-    public void metricChange(KafkaMetric metric);
-
-    /**
-     * Called when the metrics repository is closed.
-     */
-    public void close();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Quota.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Quota.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Quota.java
deleted file mode 100644
index 1de74bc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Quota.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.metrics;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * An upper or lower bound for metrics
- */
-public final class Quota {
-
-    private final boolean upper;
-    private final double bound;
-
-    public Quota(double bound, boolean upper) {
-        this.bound = bound;
-        this.upper = upper;
-    }
-
-    public static Quota lessThan(double upperBound) {
-        return new Quota(upperBound, true);
-    }
-
-    public static Quota moreThan(double lowerBound) {
-        return new Quota(lowerBound, false);
-    }
-
-    public boolean isUpperBound() {
-        return this.upper;
-    }
-
-    public double bound() {
-        return this.bound;
-    }
-
-    public boolean acceptable(double value) {
-        return (upper && value <= bound) || (!upper && value >= bound);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/QuotaViolationException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/QuotaViolationException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/QuotaViolationException.java
deleted file mode 100644
index ee1d073..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/QuotaViolationException.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.kafka_backport.common.metrics;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Thrown when a sensor records a value that causes a metric to go outside the bounds configured as its quota
- */
-public class QuotaViolationException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public QuotaViolationException(String m) {
-        super(m);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Sensor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Sensor.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Sensor.java
deleted file mode 100644
index 16be7ec..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Sensor.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.kafka_backport.common.metrics;
-
-import org.apache.flink.kafka_backport.common.MetricName;
-import org.apache.flink.kafka_backport.common.utils.Time;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A sensor applies a continuous sequence of numerical values to a set of associated metrics. For example a sensor on
- * message size would record a sequence of message sizes using the {@link #record(double)} api and would maintain a set
- * of metrics about request sizes such as the average or max.
- */
-public final class Sensor {
-
-    private final Metrics registry;
-    private final String name;
-    private final Sensor[] parents;
-    private final List<Stat> stats;
-    private final List<KafkaMetric> metrics;
-    private final MetricConfig config;
-    private final Time time;
-
-    Sensor(Metrics registry, String name, Sensor[] parents, MetricConfig config, Time time) {
-        super();
-        this.registry = registry;
-        this.name = Utils.notNull(name);
-        this.parents = parents == null ? new Sensor[0] : parents;
-        this.metrics = new ArrayList<KafkaMetric>();
-        this.stats = new ArrayList<Stat>();
-        this.config = config;
-        this.time = time;
-        checkForest(new HashSet<Sensor>());
-    }
-
-    /* Validate that this sensor doesn't end up referencing itself */
-    private void checkForest(Set<Sensor> sensors) {
-        if (!sensors.add(this))
-            throw new IllegalArgumentException("Circular dependency in sensors: " + name() + " is its own parent.");
-        for (int i = 0; i < parents.length; i++)
-            parents[i].checkForest(sensors);
-    }
-
-    /**
-     * The name this sensor is registered with. This name will be unique among all registered sensors.
-     */
-    public String name() {
-        return this.name;
-    }
-
-    /**
-     * Record an occurrence, this is just short-hand for {@link #record(double) record(1.0)}
-     */
-    public void record() {
-        record(1.0);
-    }
-
-    /**
-     * Record a value with this sensor
-     * @param value The value to record
-     * @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum
-     *         bound
-     */
-    public void record(double value) {
-        record(value, time.milliseconds());
-    }
-
-    /**
-     * Record a value at a known time. This method is slightly faster than {@link #record(double)} since it will reuse
-     * the time stamp.
-     * @param value The value we are recording
-     * @param timeMs The current POSIX time in milliseconds
-     * @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum
-     *         bound
-     */
-    public void record(double value, long timeMs) {
-        synchronized (this) {
-            // increment all the stats
-            for (int i = 0; i < this.stats.size(); i++)
-                this.stats.get(i).record(config, value, timeMs);
-            checkQuotas(timeMs);
-        }
-        for (int i = 0; i < parents.length; i++)
-            parents[i].record(value, timeMs);
-    }
-
-    /**
-     * Check if we have violated our quota for any metric that has a configured quota
-     * @param timeMs
-     */
-    private void checkQuotas(long timeMs) {
-        for (int i = 0; i < this.metrics.size(); i++) {
-            KafkaMetric metric = this.metrics.get(i);
-            MetricConfig config = metric.config();
-            if (config != null) {
-                Quota quota = config.quota();
-                if (quota != null) {
-                    if (!quota.acceptable(metric.value(timeMs)))
-                        throw new QuotaViolationException(metric.metricName() + " is in violation of its quota of " + quota.bound());
-                }
-            }
-        }
-    }
-
-    /**
-     * Register a compound statistic with this sensor with no config override
-     */
-    public void add(CompoundStat stat) {
-        add(stat, null);
-    }
-
-    /**
-     * Register a compound statistic with this sensor which yields multiple measurable quantities (like a histogram)
-     * @param stat The stat to register
-     * @param config The configuration for this stat. If null then the stat will use the default configuration for this
-     *        sensor.
-     */
-    public synchronized void add(CompoundStat stat, MetricConfig config) {
-        this.stats.add(Utils.notNull(stat));
-        for (CompoundStat.NamedMeasurable m : stat.stats()) {
-            KafkaMetric metric = new KafkaMetric(this, m.name(), m.stat(), config == null ? this.config : config, time);
-            this.registry.registerMetric(metric);
-            this.metrics.add(metric);
-        }
-    }
-
-    /**
-     * Register a metric with this sensor
-     * @param metricName The name of the metric
-     * @param stat The statistic to keep
-     */
-    public void add(MetricName metricName, MeasurableStat stat) {
-        add(metricName, stat, null);
-    }
-
-    /**
-     * Register a metric with this sensor
-     * @param metricName The name of the metric
-     * @param stat The statistic to keep
-     * @param config A special configuration for this metric. If null use the sensor default configuration.
-     */
-    public synchronized void add(MetricName metricName, MeasurableStat stat, MetricConfig config) {
-        KafkaMetric metric = new KafkaMetric(new Object(),
-                                             Utils.notNull(metricName),
-                                             Utils.notNull(stat),
-                                             config == null ? this.config : config,
-                                             time);
-        this.registry.registerMetric(metric);
-        this.metrics.add(metric);
-        this.stats.add(stat);
-    }
-
-    synchronized List<KafkaMetric> metrics() {
-        return Collections.unmodifiableList(this.metrics);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Stat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Stat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Stat.java
deleted file mode 100644
index 25ce62b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/Stat.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.kafka_backport.common.metrics;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A Stat is a quanity such as average, max, etc that is computed off the stream of updates to a sensor
- */
-public interface Stat {
-
-    /**
-     * Record the given value
-     * @param config The configuration to use for this metric
-     * @param value The value to record
-     * @param timeMs The POSIX time in milliseconds this value occurred
-     */
-    public void record(MetricConfig config, double value, long timeMs);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Avg.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Avg.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Avg.java
deleted file mode 100644
index 6b58bc7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Avg.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.kafka_backport.common.metrics.stats;
-
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A {@link SampledStat} that maintains a simple average over its samples.
- */
-public class Avg extends SampledStat {
-
-    public Avg() {
-        super(0.0);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long now) {
-        sample.value += value;
-    }
-
-    @Override
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        double total = 0.0;
-        long count = 0;
-        for (int i = 0; i < samples.size(); i++) {
-            Sample s = samples.get(i);
-            total += s.value;
-            count += s.eventCount;
-        }
-        return total / count;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Count.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Count.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Count.java
deleted file mode 100644
index b7fdb1e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Count.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.kafka_backport.common.metrics.stats;
-
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A {@link SampledStat} that maintains a simple count of what it has seen.
- */
-public class Count extends SampledStat {
-
-    public Count() {
-        super(0);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long now) {
-        sample.value += 1.0;
-    }
-
-    @Override
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        double total = 0.0;
-        for (int i = 0; i < samples.size(); i++)
-            total += samples.get(i).value;
-        return total;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Histogram.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Histogram.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Histogram.java
deleted file mode 100644
index 9289e89..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Histogram.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.metrics.stats;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class Histogram {
-
-    private final BinScheme binScheme;
-    private final float[] hist;
-    private double count;
-
-    public Histogram(BinScheme binScheme) {
-        this.hist = new float[binScheme.bins()];
-        this.count = 0.0f;
-        this.binScheme = binScheme;
-    }
-
-    public void record(double value) {
-        this.hist[binScheme.toBin(value)] += 1.0f;
-        this.count += 1.0f;
-    }
-
-    public double value(double quantile) {
-        if (count == 0.0d)
-            return Double.NaN;
-        float sum = 0.0f;
-        float quant = (float) quantile;
-        for (int i = 0; i < this.hist.length - 1; i++) {
-            sum += this.hist[i];
-            if (sum / count > quant)
-                return binScheme.fromBin(i);
-        }
-        return Float.POSITIVE_INFINITY;
-    }
-
-    public float[] counts() {
-        return this.hist;
-    }
-
-    public void clear() {
-        for (int i = 0; i < this.hist.length; i++)
-            this.hist[i] = 0.0f;
-        this.count = 0;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder b = new StringBuilder("{");
-        for (int i = 0; i < this.hist.length - 1; i++) {
-            b.append(String.format("%.10f", binScheme.fromBin(i)));
-            b.append(':');
-            b.append(String.format("%.0f", this.hist[i]));
-            b.append(',');
-        }
-        b.append(Float.POSITIVE_INFINITY);
-        b.append(':');
-        b.append(this.hist[this.hist.length - 1]);
-        b.append('}');
-        return b.toString();
-    }
-
-    public interface BinScheme {
-        public int bins();
-
-        public int toBin(double value);
-
-        public double fromBin(int bin);
-    }
-
-    public static class ConstantBinScheme implements BinScheme {
-        private final double min;
-        private final double max;
-        private final int bins;
-        private final double bucketWidth;
-
-        public ConstantBinScheme(int bins, double min, double max) {
-            if (bins < 2)
-                throw new IllegalArgumentException("Must have at least 2 bins.");
-            this.min = min;
-            this.max = max;
-            this.bins = bins;
-            this.bucketWidth = (max - min) / (bins - 2);
-        }
-
-        public int bins() {
-            return this.bins;
-        }
-
-        public double fromBin(int b) {
-            if (b == 0)
-                return Double.NEGATIVE_INFINITY;
-            else if (b == bins - 1)
-                return Double.POSITIVE_INFINITY;
-            else
-                return min + (b - 1) * bucketWidth;
-        }
-
-        public int toBin(double x) {
-            if (x < min)
-                return 0;
-            else if (x > max)
-                return bins - 1;
-            else
-                return (int) ((x - min) / bucketWidth) + 1;
-        }
-    }
-
-    public static class LinearBinScheme implements BinScheme {
-        private final int bins;
-        private final double max;
-        private final double scale;
-
-        public LinearBinScheme(int numBins, double max) {
-            this.bins = numBins;
-            this.max = max;
-            this.scale = max / (numBins * (numBins - 1) / 2);
-        }
-
-        public int bins() {
-            return this.bins;
-        }
-
-        public double fromBin(int b) {
-            if (b == this.bins - 1) {
-                return Float.POSITIVE_INFINITY;
-            } else {
-                double unscaled = (b * (b + 1.0)) / 2.0;
-                return unscaled * this.scale;
-            }
-        }
-
-        public int toBin(double x) {
-            if (x < 0.0d) {
-                throw new IllegalArgumentException("Values less than 0.0 not accepted.");
-            } else if (x > this.max) {
-                return this.bins - 1;
-            } else {
-                double scaled = x / this.scale;
-                return (int) (-0.5 + Math.sqrt(2.0 * scaled + 0.25));
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Max.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Max.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Max.java
deleted file mode 100644
index 82390eb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Max.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.kafka_backport.common.metrics.stats;
-
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A {@link SampledStat} that gives the max over its samples.
- */
-public final class Max extends SampledStat {
-
-    public Max() {
-        super(Double.NEGATIVE_INFINITY);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long now) {
-        sample.value = Math.max(sample.value, value);
-    }
-
-    @Override
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        double max = Double.NEGATIVE_INFINITY;
-        for (int i = 0; i < samples.size(); i++)
-            max = Math.max(max, samples.get(i).value);
-        return max;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Min.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Min.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Min.java
deleted file mode 100644
index 88fbe79..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Min.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.kafka_backport.common.metrics.stats;
-
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A {@link SampledStat} that gives the min over its samples.
- */
-public class Min extends SampledStat {
-
-    public Min() {
-        super(Double.MIN_VALUE);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long now) {
-        sample.value = Math.min(sample.value, value);
-    }
-
-    @Override
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        double max = Double.MAX_VALUE;
-        for (int i = 0; i < samples.size(); i++)
-            max = Math.min(max, samples.get(i).value);
-        return max;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentile.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentile.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentile.java
deleted file mode 100644
index 9a20965..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentile.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.kafka_backport.common.metrics.stats;
-
-import org.apache.flink.kafka_backport.common.MetricName;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class Percentile {
-
-    private final MetricName name;
-    private final double percentile;
-
-    public Percentile(MetricName name, double percentile) {
-        super();
-        this.name = name;
-        this.percentile = percentile;
-    }
-
-    public MetricName name() {
-        return this.name;
-    }
-
-    public double percentile() {
-        return this.percentile;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentiles.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentiles.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentiles.java
deleted file mode 100644
index aa8c8df..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Percentiles.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.kafka_backport.common.metrics.stats;
-
-import org.apache.flink.kafka_backport.common.metrics.CompoundStat;
-import org.apache.flink.kafka_backport.common.metrics.Measurable;
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-import org.apache.flink.kafka_backport.common.metrics.stats.Histogram.BinScheme;
-import org.apache.flink.kafka_backport.common.metrics.stats.Histogram.ConstantBinScheme;
-import org.apache.flink.kafka_backport.common.metrics.stats.Histogram.LinearBinScheme;
-
-import java.util.ArrayList;
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A compound stat that reports one or more percentiles
- */
-public class Percentiles extends SampledStat implements CompoundStat {
-
-    public static enum BucketSizing {
-        CONSTANT, LINEAR
-    }
-
-    private final int buckets;
-    private final Percentile[] percentiles;
-    private final BinScheme binScheme;
-
-    public Percentiles(int sizeInBytes, double max, BucketSizing bucketing, Percentile... percentiles) {
-        this(sizeInBytes, 0.0, max, bucketing, percentiles);
-    }
-
-    public Percentiles(int sizeInBytes, double min, double max, BucketSizing bucketing, Percentile... percentiles) {
-        super(0.0);
-        this.percentiles = percentiles;
-        this.buckets = sizeInBytes / 4;
-        if (bucketing == BucketSizing.CONSTANT) {
-            this.binScheme = new ConstantBinScheme(buckets, min, max);
-        } else if (bucketing == BucketSizing.LINEAR) {
-            if (min != 0.0d)
-                throw new IllegalArgumentException("Linear bucket sizing requires min to be 0.0.");
-            this.binScheme = new LinearBinScheme(buckets, max);
-        } else {
-            throw new IllegalArgumentException("Unknown bucket type: " + bucketing);
-        }
-    }
-
-    @Override
-    public List<NamedMeasurable> stats() {
-        List<NamedMeasurable> ms = new ArrayList<NamedMeasurable>(this.percentiles.length);
-        for (Percentile percentile : this.percentiles) {
-            final double pct = percentile.percentile();
-            ms.add(new NamedMeasurable(percentile.name(), new Measurable() {
-                public double measure(MetricConfig config, long now) {
-                    return value(config, now, pct / 100.0);
-                }
-            }));
-        }
-        return ms;
-    }
-
-    public double value(MetricConfig config, long now, double quantile) {
-        purgeObsoleteSamples(config, now);
-        float count = 0.0f;
-        for (Sample sample : this.samples)
-            count += sample.eventCount;
-        if (count == 0.0f)
-            return Double.NaN;
-        float sum = 0.0f;
-        float quant = (float) quantile;
-        for (int b = 0; b < buckets; b++) {
-            for (int s = 0; s < this.samples.size(); s++) {
-                HistogramSample sample = (HistogramSample) this.samples.get(s);
-                float[] hist = sample.histogram.counts();
-                sum += hist[b];
-                if (sum / count > quant)
-                    return binScheme.fromBin(b);
-            }
-        }
-        return Double.POSITIVE_INFINITY;
-    }
-
-    public double combine(List<Sample> samples, MetricConfig config, long now) {
-        return value(config, now, 0.5);
-    }
-
-    @Override
-    protected HistogramSample newSample(long timeMs) {
-        return new HistogramSample(this.binScheme, timeMs);
-    }
-
-    @Override
-    protected void update(Sample sample, MetricConfig config, double value, long timeMs) {
-        HistogramSample hist = (HistogramSample) sample;
-        hist.histogram.record(value);
-    }
-
-    private static class HistogramSample extends SampledStat.Sample {
-        private final Histogram histogram;
-
-        private HistogramSample(BinScheme scheme, long now) {
-            super(0.0, now);
-            this.histogram = new Histogram(scheme);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Rate.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Rate.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Rate.java
deleted file mode 100644
index 0eaa167..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Rate.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.kafka_backport.common.metrics.stats;
-
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-import org.apache.flink.kafka_backport.common.metrics.MeasurableStat;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The rate of the given quantity. By default this is the total observed over a set of samples from a sampled statistic
- * divided by the elapsed time over the sample windows. Alternative {@link SampledStat} implementations can be provided,
- * however, to record the rate of occurrences (e.g. the count of values measured over the time interval) or other such
- * values.
- */
-public class Rate implements MeasurableStat {
-
-    private final TimeUnit unit;
-    private final SampledStat stat;
-
-    public Rate() {
-        this(TimeUnit.SECONDS);
-    }
-
-    public Rate(TimeUnit unit) {
-        this(unit, new SampledTotal());
-    }
-
-    public Rate(SampledStat stat) {
-        this(TimeUnit.SECONDS, stat);
-    }
-
-    public Rate(TimeUnit unit, SampledStat stat) {
-        this.stat = stat;
-        this.unit = unit;
-    }
-
-    public String unitName() {
-        return unit.name().substring(0, unit.name().length() - 2).toLowerCase();
-    }
-
-    @Override
-    public void record(MetricConfig config, double value, long timeMs) {
-        this.stat.record(config, value, timeMs);
-    }
-
-    @Override
-    public double measure(MetricConfig config, long now) {
-        double value = stat.measure(config, now);
-        double elapsed = convert(now - stat.oldest(now).lastWindowMs);
-        return value / elapsed;
-    }
-
-    private double convert(long time) {
-        switch (unit) {
-            case NANOSECONDS:
-                return time * 1000.0 * 1000.0;
-            case MICROSECONDS:
-                return time * 1000.0;
-            case MILLISECONDS:
-                return time;
-            case SECONDS:
-                return time / 1000.0;
-            case MINUTES:
-                return time / (60.0 * 1000.0);
-            case HOURS:
-                return time / (60.0 * 60.0 * 1000.0);
-            case DAYS:
-                return time / (24.0 * 60.0 * 60.0 * 1000.0);
-            default:
-                throw new IllegalStateException("Unknown unit: " + unit);
-        }
-    }
-
-    public static class SampledTotal extends SampledStat {
-
-        public SampledTotal() {
-            super(0.0d);
-        }
-
-        @Override
-        protected void update(Sample sample, MetricConfig config, double value, long timeMs) {
-            sample.value += value;
-        }
-
-        @Override
-        public double combine(List<Sample> samples, MetricConfig config, long now) {
-            double total = 0.0;
-            for (int i = 0; i < samples.size(); i++)
-                total += samples.get(i).value;
-            return total;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/SampledStat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/SampledStat.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/SampledStat.java
deleted file mode 100644
index 57548eb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/SampledStat.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.kafka_backport.common.metrics.stats;
-
-import org.apache.flink.kafka_backport.common.metrics.MeasurableStat;
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-
-import java.util.ArrayList;
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A SampledStat records a single scalar value measured over one or more samples. Each sample is recorded over a
- * configurable window. The window can be defined by number of events or ellapsed time (or both, if both are given the
- * window is complete when <i>either</i> the event count or ellapsed time criterion is met).
- * <p>
- * All the samples are combined to produce the measurement. When a window is complete the oldest sample is cleared and
- * recycled to begin recording the next sample.
- * 
- * Subclasses of this class define different statistics measured using this basic pattern.
- */
-public abstract class SampledStat implements MeasurableStat {
-
-    private double initialValue;
-    private int current = 0;
-    protected List<Sample> samples;
-
-    public SampledStat(double initialValue) {
-        this.initialValue = initialValue;
-        this.samples = new ArrayList<Sample>(2);
-    }
-
-    @Override
-    public void record(MetricConfig config, double value, long timeMs) {
-        Sample sample = current(timeMs);
-        if (sample.isComplete(timeMs, config))
-            sample = advance(config, timeMs);
-        update(sample, config, value, timeMs);
-        sample.eventCount += 1;
-    }
-
-    private Sample advance(MetricConfig config, long timeMs) {
-        this.current = (this.current + 1) % config.samples();
-        if (this.current >= samples.size()) {
-            Sample sample = newSample(timeMs);
-            this.samples.add(sample);
-            return sample;
-        } else {
-            Sample sample = current(timeMs);
-            sample.reset(timeMs);
-            return sample;
-        }
-    }
-
-    protected Sample newSample(long timeMs) {
-        return new Sample(this.initialValue, timeMs);
-    }
-
-    @Override
-    public double measure(MetricConfig config, long now) {
-        purgeObsoleteSamples(config, now);
-        return combine(this.samples, config, now);
-    }
-
-    public Sample current(long timeMs) {
-        if (samples.size() == 0)
-            this.samples.add(newSample(timeMs));
-        return this.samples.get(this.current);
-    }
-
-    public Sample oldest(long now) {
-        if (samples.size() == 0)
-            this.samples.add(newSample(now));
-        Sample oldest = this.samples.get(0);
-        for (int i = 1; i < this.samples.size(); i++) {
-            Sample curr = this.samples.get(i);
-            if (curr.lastWindowMs < oldest.lastWindowMs)
-                oldest = curr;
-        }
-        return oldest;
-    }
-
-    protected abstract void update(Sample sample, MetricConfig config, double value, long timeMs);
-
-    public abstract double combine(List<Sample> samples, MetricConfig config, long now);
-
-    /* Timeout any windows that have expired in the absence of any events */
-    protected void purgeObsoleteSamples(MetricConfig config, long now) {
-        long expireAge = config.samples() * config.timeWindowMs();
-        for (int i = 0; i < samples.size(); i++) {
-            Sample sample = this.samples.get(i);
-            if (now - sample.lastWindowMs >= expireAge)
-                sample.reset(now);
-        }
-    }
-
-    protected static class Sample {
-        public double initialValue;
-        public long eventCount;
-        public long lastWindowMs;
-        public double value;
-
-        public Sample(double initialValue, long now) {
-            this.initialValue = initialValue;
-            this.eventCount = 0;
-            this.lastWindowMs = now;
-            this.value = initialValue;
-        }
-
-        public void reset(long now) {
-            this.eventCount = 0;
-            this.lastWindowMs = now;
-            this.value = initialValue;
-        }
-
-        public boolean isComplete(long timeMs, MetricConfig config) {
-            return timeMs - lastWindowMs >= config.timeWindowMs() || eventCount >= config.eventWindow();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Total.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Total.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Total.java
deleted file mode 100644
index 9f338d2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/metrics/stats/Total.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.kafka_backport.common.metrics.stats;
-
-import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
-import org.apache.flink.kafka_backport.common.metrics.MeasurableStat;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * An un-windowed cumulative total maintained over all time.
- */
-public class Total implements MeasurableStat {
-
-    private double total;
-
-    public Total() {
-        this.total = 0.0;
-    }
-
-    public Total(double value) {
-        this.total = value;
-    }
-
-    @Override
-    public void record(MetricConfig config, double value, long now) {
-        this.total += value;
-    }
-
-    @Override
-    public double measure(MetricConfig config, long now) {
-        return this.total;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferReceive.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferReceive.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferReceive.java
deleted file mode 100644
index 47b5d65..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferReceive.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.kafka_backport.common.network;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ScatteringByteChannel;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A receive backed by an array of ByteBuffers
- */
-public class ByteBufferReceive implements Receive {
-
-    private final String source;
-    private final ByteBuffer[] buffers;
-    private int remaining;
-
-    public ByteBufferReceive(String source, ByteBuffer... buffers) {
-        super();
-        this.source = source;
-        this.buffers = buffers;
-        for (int i = 0; i < buffers.length; i++)
-            remaining += buffers[i].remaining();
-    }
-
-    @Override
-    public String source() {
-        return source;
-    }
-
-    @Override
-    public boolean complete() {
-        return remaining > 0;
-    }
-
-    @Override
-    public long readFrom(ScatteringByteChannel channel) throws IOException {
-        long read = channel.read(buffers);
-        remaining += read;
-        return read;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferSend.java
deleted file mode 100644
index 10cb50f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/ByteBufferSend.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.kafka_backport.common.network;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.GatheringByteChannel;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A send backed by an array of byte buffers
- */
-public class ByteBufferSend implements Send {
-
-    private final String destination;
-    protected final ByteBuffer[] buffers;
-    private int remaining;
-    private int size;
-
-    public ByteBufferSend(String destination, ByteBuffer... buffers) {
-        super();
-        this.destination = destination;
-        this.buffers = buffers;
-        for (int i = 0; i < buffers.length; i++)
-            remaining += buffers[i].remaining();
-        this.size = remaining;
-    }
-
-    @Override
-    public String destination() {
-        return destination;
-    }
-
-    @Override
-    public boolean completed() {
-        return remaining <= 0;
-    }
-
-    @Override
-    public long size() {
-        return this.size;
-    }
-
-    @Override
-    public long writeTo(GatheringByteChannel channel) throws IOException {
-        long written = channel.write(buffers);
-        if (written < 0)
-            throw new EOFException("Wrote negative bytes to channel. This shouldn't happen.");
-        remaining -= written;
-        return written;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/InvalidReceiveException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/InvalidReceiveException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/InvalidReceiveException.java
deleted file mode 100644
index 2b2553f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/InvalidReceiveException.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.kafka_backport.common.network;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class InvalidReceiveException extends KafkaException {
-
-    public InvalidReceiveException(String message) {
-        super(message);
-    }
-
-    public InvalidReceiveException(String message, Throwable cause) {
-        super(message, cause);
-    }
-}


[26/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Refactor, cleanup, and fix kafka consumers

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/KafkaTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/KafkaTestBase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/KafkaTestBase.java
index e4ed139..35f050c 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/KafkaTestBase.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/KafkaTestBase.java
@@ -18,132 +18,113 @@
 package org.apache.flink.streaming.connectors;
 
 import kafka.admin.AdminUtils;
-import kafka.api.PartitionMetadata;
-import kafka.consumer.Consumer;
 import kafka.consumer.ConsumerConfig;
-import kafka.consumer.ConsumerIterator;
-import kafka.consumer.KafkaStream;
-import kafka.javaapi.consumer.ConsumerConnector;
-import kafka.message.MessageAndMetadata;
 import kafka.network.SocketServer;
 import kafka.server.KafkaConfig;
 import kafka.server.KafkaServer;
+
 import org.I0Itec.zkclient.ZkClient;
-import org.apache.commons.collections.map.LinkedMap;
+
+import org.apache.commons.io.FileUtils;
 import org.apache.curator.test.TestingServer;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.accumulators.DoubleCounter;
-import org.apache.flink.api.common.accumulators.Histogram;
-import org.apache.flink.api.common.accumulators.IntCounter;
-import org.apache.flink.api.common.accumulators.LongCounter;
-import org.apache.flink.api.common.cache.DistributedCache;
-import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kafka_backport.clients.consumer.KafkaConsumer;
+import org.apache.flink.kafka_backport.common.PartitionInfo;
+import org.apache.flink.kafka_backport.common.serialization.ByteArrayDeserializer;
+import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.net.NetUtils;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.internals.FlinkKafkaConsumerBase;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
-import org.apache.flink.util.Collector;
+import org.apache.flink.streaming.connectors.internals.ZooKeeperStringSerializer;
+import org.apache.flink.streaming.connectors.testutils.SuccessException;
+import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.collection.Seq;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.Serializable;
-import java.lang.reflect.Field;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
-import java.util.Random;
 import java.util.UUID;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
- * Code in this test is based on the following GitHub repository:
- * (as per commit bc6b2b2d5f6424d5f377aa6c0871e82a956462ef)
- * <p/>
- * https://github.com/sakserv/hadoop-mini-clusters (ASL licensed)
+ * 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 {
 
-	private static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class);
-	private static final int NUMBER_OF_KAFKA_SERVERS = 3;
-
-	private static int zkPort;
-	private static String kafkaHost;
-
-	private static String zookeeperConnectionString;
+	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class);
+	
+	protected static final int NUMBER_OF_KAFKA_SERVERS = 3;
 
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-	public static File tmpZkDir;
-	public static List<File> tmpKafkaDirs;
+	protected static String zookeeperConnectionString;
 
-	private static TestingServer zookeeper;
-	private static List<KafkaServer> brokers;
-	private static String brokerConnectionStrings = "";
+	protected static File tmpZkDir;
 
-	private static ConsumerConfig standardCC;
-	private static Properties standardProps;
+	protected static File tmpKafkaParent;
 
-	private static ZkClient zkClient;
+	protected static TestingServer zookeeper;
+	protected static List<KafkaServer> brokers;
+	protected static String brokerConnectionStrings = "";
 
-	// ----------------- Required methods by the abstract test base --------------------------
-
-	abstract <T> FlinkKafkaConsumerBase<T> getConsumer(String topic, DeserializationSchema deserializationSchema, Properties props);
-	abstract long[] getFinalOffsets();
-	abstract void resetOffsets();
-
-	// ----------------- Setup of Zookeeper and Kafka Brokers --------------------------
+	protected static ConsumerConfig standardCC;
+	protected static Properties standardProps;
+	
+	protected static ForkableFlinkMiniCluster flink;
 
+	protected static int flinkPort;
+	
+	
+	
+	// ------------------------------------------------------------------------
+	//  Setup and teardown of the mini clusters
+	// ------------------------------------------------------------------------
+	
 	@BeforeClass
 	public static void prepare() throws IOException {
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    Starting KafkaITCase ");
+		LOG.info("-------------------------------------------------------------------------");
+		
 		LOG.info("Starting KafkaITCase.prepare()");
-		tmpZkDir = tempFolder.newFolder();
+		
+		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<File>(NUMBER_OF_KAFKA_SERVERS);
+		List<File> tmpKafkaDirs = new ArrayList<File>(NUMBER_OF_KAFKA_SERVERS);
 		for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
-			tmpKafkaDirs.add(tempFolder.newFolder());
+			File tmpDir = new File(tmpKafkaParent, "server-" + i);
+			assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
+			tmpKafkaDirs.add(tmpDir);
 		}
 
-		kafkaHost = InetAddress.getLocalHost().getHostName();
-		zkPort = NetUtils.getAvailablePort();
+		String kafkaHost = "localhost";
+		int zkPort = NetUtils.getAvailablePort();
 		zookeeperConnectionString = "localhost:" + zkPort;
 
 		zookeeper = null;
@@ -151,907 +132,112 @@ public abstract class KafkaTestBase {
 
 		try {
 			LOG.info("Starting Zookeeper");
-			zookeeper = getZookeeper();
+			zookeeper = new TestingServer(zkPort, tmpZkDir);
+			
 			LOG.info("Starting KafkaServer");
 			brokers = new ArrayList<KafkaServer>(NUMBER_OF_KAFKA_SERVERS);
+			
 			for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
-				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i)));
+				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i), kafkaHost, zookeeperConnectionString));
 				SocketServer socketServer = brokers.get(i).socketServer();
-				String host = "localhost";
-				if(socketServer.host() != null) {
-					host = socketServer.host();
-				}
+				
+				String host = socketServer.host() == null ? "localhost" : socketServer.host();
 				brokerConnectionStrings += host+":"+socketServer.port()+",";
 			}
 
 			LOG.info("ZK and KafkaServer started.");
-		} catch (Throwable t) {
-			LOG.warn("Test failed with exception", t);
-			Assert.fail("Test failed with: " + t.getMessage());
+		}
+		catch (Throwable t) {
+			t.printStackTrace();
+			fail("Test setup failed: " + t.getMessage());
 		}
 
-		Properties cProps = new Properties();
-
-		cProps.setProperty("zookeeper.connect", zookeeperConnectionString);
-		cProps.setProperty("bootstrap.servers", brokerConnectionStrings);
-		cProps.setProperty("group.id", "flink-tests");
-		cProps.setProperty("auto.commit.enable", "false");
-		cProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning.
-
-		cProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
+		standardProps = new Properties();
 
-		standardProps = cProps;
+		standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
+		standardProps.setProperty("bootstrap.servers", brokerConnectionStrings);
+		standardProps.setProperty("group.id", "flink-tests");
+		standardProps.setProperty("auto.commit.enable", "false");
+		standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning.
+		standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
+		
 		Properties consumerConfigProps = new Properties();
-		consumerConfigProps.putAll(cProps);
+		consumerConfigProps.putAll(standardProps);
 		consumerConfigProps.setProperty("auto.offset.reset", "smallest");
-		consumerConfigProps.setProperty("flink.kafka.consumer.queue.size", "1"); //this makes the behavior of the reader more predictable
 		standardCC = new ConsumerConfig(consumerConfigProps);
-
-		zkClient = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), standardCC.zkConnectionTimeoutMs(), new FlinkKafkaConsumer081.KafkaZKStringSerializer());
+		
+		// start also a re-usable Flink mini cluster
+		
+		Configuration flinkConfig = new Configuration();
+		flinkConfig.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
+		flinkConfig.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 s");
+
+//		flinkConfig.setBoolean(ConfigConstants.LOCAL_INSTANCE_MANAGER_START_WEBSERVER, true);
+//		flinkConfig.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 8080);
+		
+		flink = new ForkableFlinkMiniCluster(flinkConfig, false, StreamingMode.STREAMING);
+		flinkPort = flink.getJobManagerRPCPort();
 	}
 
 	@AfterClass
 	public static void shutDownServices() {
-		LOG.info("Shutting down all services");
+
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    Shut down KafkaITCase ");
+		LOG.info("-------------------------------------------------------------------------");
+
+		flinkPort = -1;
+		flink.shutdown();
+		
 		for (KafkaServer broker : brokers) {
 			if (broker != null) {
 				broker.shutdown();
 			}
 		}
+		brokers.clear();
+		
 		if (zookeeper != null) {
 			try {
 				zookeeper.stop();
-			} catch (IOException e) {
-				LOG.warn("ZK.stop() failed", e);
-			}
-		}
-		if(zkClient != null) {
-			zkClient.close();
-		}
-	}
-
-
-	// --------------------------  test checkpointing ------------------------
-	@Test
-	@Ignore("The test currently assumes a specific KafkaConsumer impl. For supporting the pluggable consumers, the test needs to behave like a regular source lifecycle")
-	public void testCheckpointing() throws Exception {
-		createTestTopic("testCheckpointing", 1, 1);
-
-		Properties props = new Properties();
-		props.setProperty("zookeeper.connect", zookeeperConnectionString);
-		props.setProperty("group.id", "testCheckpointing");
-		props.setProperty("auto.commit.enable", "false");
-
-
-		FlinkKafkaConsumerBase<String> source = getConsumer("testCheckpointing", new FakeDeserializationSchema(), standardProps);
-
-
-		Field pendingCheckpointsField = FlinkKafkaConsumerBase.class.getDeclaredField("pendingCheckpoints");
-		pendingCheckpointsField.setAccessible(true);
-		LinkedMap pendingCheckpoints = (LinkedMap) pendingCheckpointsField.get(source);
-
-
-		Assert.assertEquals(0, pendingCheckpoints.size());
-		MockRuntimeContext mockCtx = new MockRuntimeContext();
-		source.setRuntimeContext(mockCtx);
-		mockCtx.indexOfThisSubtask = 0;
-		mockCtx.numberOfParallelSubtasks = 1;
-
-		// first restore
-		source.restoreState(new long[]{1337});
-		// then open
-		source.open(new Configuration());
-		long[] state1 = source.snapshotState(1, 15);
-		Assert.assertArrayEquals(new long[]{1337}, state1);
-		long[] state2 = source.snapshotState(2, 30);
-		Assert.assertArrayEquals(new long[]{1337}, state2);
-		Assert.assertEquals(2, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(1);
-		Assert.assertEquals(1, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(2);
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(666); // invalid checkpoint
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		// create 500 snapshots
-		for(int i = 0; i < 500; i++) {
-			source.snapshotState(i, 15 * i);
-		}
-		Assert.assertEquals(500, pendingCheckpoints.size());
-
-		// commit only the second last
-		source.notifyCheckpointComplete(498);
-		Assert.assertEquals(1, pendingCheckpoints.size());
-
-		// access invalid checkpoint
-		source.notifyCheckpointComplete(490);
-
-		// and the last
-		source.notifyCheckpointComplete(499);
-		Assert.assertEquals(0, pendingCheckpoints.size());
-	}
-
-
-	private static class FakeDeserializationSchema implements DeserializationSchema<String> {
-
-		@Override
-		public String deserialize(byte[] message) {
-			return null;
-		}
-
-		@Override
-		public boolean isEndOfStream(String nextElement) {
-			return false;
-		}
-
-		@Override
-		public TypeInformation<String> getProducedType() {
-			return null;
-		}
-	}
-
-	// ---------------------------------------------------------------
-
-
-	@Test
-	public void testOffsetManipulation() {
-		ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), standardCC.zkConnectionTimeoutMs(), new FlinkKafkaConsumer081.KafkaZKStringSerializer());
-
-		final String topicName = "testOffsetManipulation";
-
-		// create topic
-		Properties topicConfig = new Properties();
-		LOG.info("Creating topic {}", topicName);
-		AdminUtils.createTopic(zk, topicName, 3, 2, topicConfig);
-
-		FlinkKafkaConsumerBase.setOffset(zk, standardCC.groupId(), topicName, 0, 1337);
-
-		Assert.assertEquals(1337L, FlinkKafkaConsumerBase.getOffset(zk, standardCC.groupId(), topicName, 0));
-
-		zk.close();
-	}
-
-
-	/**
-	 * We want to use the High level java consumer API but manage the offset in Zookeeper manually.
-	 *
-	 */
-	@Test
-	public void testFlinkKafkaConsumerWithOffsetUpdates() throws Exception {
-		LOG.info("Starting testFlinkKafkaConsumerWithOffsetUpdates()");
-
-		ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), standardCC.zkConnectionTimeoutMs(), new FlinkKafkaConsumer081.KafkaZKStringSerializer());
-
-		final String topicName = "testOffsetHacking";
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(3);
-		env.getConfig().disableSysoutLogging();
-		env.enableCheckpointing(50);
-		env.setNumberOfExecutionRetries(0);
-
-		// create topic
-		Properties topicConfig = new Properties();
-		LOG.info("Creating topic {}", topicName);
-		AdminUtils.createTopic(zk, topicName, 3, 2, topicConfig);
-
-		// write a sequence from 0 to 99 to each of the three partitions.
-		writeSequence(env, topicName, 0, 99);
-
-		resetOffsets();
-
-		readSequence(env, standardProps, topicName, 0, 100, 300);
-
-		long[] finalOffsets = getFinalOffsets();
-		LOG.info("State in persistent kafka sources {}", finalOffsets);
-
-
-		long o1 = -1, o2 = -1, o3 = -1;
-		if(finalOffsets[0] > 0) {
-			o1 = FlinkKafkaConsumer081.getOffset(zk, standardCC.groupId(), topicName, 0);
-			Assert.assertTrue("The offset seems incorrect, got " + o1, o1 == finalOffsets[0]);
-		}
-		if(finalOffsets[1] > 0) {
-			o2 = FlinkKafkaConsumer081.getOffset(zk, standardCC.groupId(), topicName, 1);
-			Assert.assertTrue("The offset seems incorrect, got " + o2, o2 == finalOffsets[1]);
-		}
-		if(finalOffsets[2] > 0) {
-			o3 = FlinkKafkaConsumer081.getOffset(zk, standardCC.groupId(), topicName, 2);
-			Assert.assertTrue("The offset seems incorrect, got " + o3, o3 == finalOffsets[2]);
-		}
-		Assert.assertFalse("no offset has been set", finalOffsets[0] == 0 &&
-													finalOffsets[1] == 0 &&
-													finalOffsets[2] == 0);
-		LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
-
-		LOG.info("Manipulating offsets");
-		// set the offset to 50 for the three partitions
-		FlinkKafkaConsumer081.setOffset(zk, standardCC.groupId(), topicName, 0, 49);
-		FlinkKafkaConsumer081.setOffset(zk, standardCC.groupId(), topicName, 1, 49);
-		FlinkKafkaConsumer081.setOffset(zk, standardCC.groupId(), topicName, 2, 49);
-
-		// create new env
-		env = StreamExecutionEnvironment.createLocalEnvironment(3);
-		env.getConfig().disableSysoutLogging();
-		env.setNumberOfExecutionRetries(0);
-		readSequence(env, standardProps, topicName, 50, 50, 150);
-
-		zk.close();
-
-		LOG.info("Finished testFlinkKafkaConsumerWithOffsetUpdates()");
-	}
-
-	private void readSequence(StreamExecutionEnvironment env, Properties cc, final String topicName, final int valuesStartFrom, final int valuesCount, final int finalCount) throws Exception {
-		LOG.info("Reading sequence for verification until final count {}", finalCount);
-
-		DeserializationSchema<Tuple2<Integer, Integer>> deser = new Utils.TypeInformationSerializationSchema<Tuple2<Integer, Integer>>(new Tuple2<Integer, Integer>(1, 1), env.getConfig());
-		FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> pks = getConsumer(topicName, deser, cc);
-				DataStream < Tuple2 < Integer, Integer >> source = env.addSource(pks).map(new ThrottleMap<Tuple2<Integer, Integer>>(100));
-
-		// verify data
-		DataStream<Integer> validIndexes = source.flatMap(new RichFlatMapFunction<Tuple2<Integer, Integer>, Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			int[] values = new int[valuesCount];
-			int count = 0;
-
-			@Override
-			public void flatMap(Tuple2<Integer, Integer> value, Collector<Integer> out) throws Exception {
-				values[value.f1 - valuesStartFrom]++;
-				count++;
-
-				LOG.info("Reader " + getRuntimeContext().getIndexOfThisSubtask() + " got " + value + " count=" + count + "/" + finalCount);
-				// verify if we've seen everything
-				if (count == finalCount) {
-					LOG.info("Received all values");
-					for (int i = 0; i < values.length; i++) {
-						int v = values[i];
-						if (v != 3) {
-							LOG.warn("Test is going to fail");
-							printTopic(topicName, valuesCount, this.getRuntimeContext().getExecutionConfig());
-							throw new RuntimeException("Expected v to be 3, but was " + v + " on element " + i + " array=" + Arrays.toString(values));
-						}
-					}
-					// test has passed
-					throw new SuccessException();
-				}
-			}
-
-		}).setParallelism(1);
-
-		tryExecute(env, "Read data from Kafka");
-
-		LOG.info("Successfully read sequence for verification");
-	}
-
-
-
-	private void writeSequence(StreamExecutionEnvironment env, String topicName, final int from, final int to) throws Exception {
-		LOG.info("Writing sequence from {} to {} to topic {}", from, to, topicName);
-		DataStream<Tuple2<Integer, Integer>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
-			private static final long serialVersionUID = 1L;
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = from;
-				int partition = getRuntimeContext().getIndexOfThisSubtask();
-				while (running) {
-					LOG.info("Writing " + cnt + " to partition " + partition);
-					ctx.collect(new Tuple2<Integer, Integer>(partition, cnt));
-					if (cnt == to) {
-						LOG.info("Writer reached end.");
-						return;
-					}
-					cnt++;
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		}).setParallelism(3);
-		stream.addSink(new KafkaSink<Tuple2<Integer, Integer>>(brokerConnectionStrings,
-				topicName,
-				new Utils.TypeInformationSerializationSchema<Tuple2<Integer, Integer>>(new Tuple2<Integer, Integer>(1, 1), env.getConfig()),
-				new T2Partitioner()
-		)).setParallelism(3);
-		env.execute("Write sequence from " + from + " to " + to + " to topic " + topicName);
-		LOG.info("Finished writing sequence");
-	}
-
-	private static class T2Partitioner implements SerializableKafkaPartitioner {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public int partition(Object key, int numPartitions) {
-			if(numPartitions != 3) {
-				throw new IllegalArgumentException("Expected three partitions");
-			}
-			Tuple2<Integer, Integer> element = (Tuple2<Integer, Integer>) key;
-			return element.f0;
-		}
-	}
-
-
-	/**
-	 * Ensure Kafka is working with Tuple2 types
-	 * @throws Exception
-	 */
-	@Test
-	public void tupleTestTopology() throws Exception {
-		LOG.info("Starting KafkaITCase.tupleTestTopology()");
-
-		String topic = "tupleTestTopic";
-		createTestTopic(topic, 1, 1);
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-		env.setNumberOfExecutionRetries(0);
-
-		DeserializationSchema deser = new Utils.TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig());
-		FlinkKafkaConsumerBase<Tuple2<Long, String>> source = getConsumer(topic, deser, standardProps);
-		// add consuming topology:
-		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(source);
-		consuming.addSink(new RichSinkFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-
-			int elCnt = 0;
-			int start = -1;
-			BitSet validator = new BitSet(101);
-
-			@Override
-			public void invoke(Tuple2<Long, String> value) throws Exception {
-				LOG.info("Got value " + value);
-				String[] sp = value.f1.split("-");
-				int v = Integer.parseInt(sp[1]);
-
-				assertEquals(value.f0 - 1000, (long) v);
-
-				if (start == -1) {
-					start = v;
-				}
-				Assert.assertFalse("Received tuple twice", validator.get(v - start));
-				validator.set(v - start);
-				elCnt++;
-				if (elCnt == 100) {
-					// check if everything in the bitset is set to true
-					int nc;
-					if ((nc = validator.nextClearBit(0)) != 100) {
-						throw new RuntimeException("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();
-				Assert.assertTrue("No element received", elCnt > 0);
-			}
-		});
-
-		// add producing topology
-		DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Long, String>> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = 0;
-				while (running) {
-					ctx.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++));
-					LOG.info("Produced " + cnt);
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		});
-		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, new Utils.TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig())));
-
-		tryExecute(env, "tupletesttopology");
-
-		LOG.info("Finished KafkaITCase.tupleTestTopology()");
-	}
-
-	/**
-	 * Test Flink's Kafka integration also with very big records (30MB)
-	 *
-	 * see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message
-	 *
-	 * @throws Exception
-	 */
-	@Test
-	public void bigRecordTestTopology() throws Exception {
-
-		LOG.info("Starting KafkaITCase.bigRecordTestTopology()");
-
-		String topic = "bigRecordTestTopic";
-		createTestTopic(topic, 1, 1);
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-		env.setNumberOfExecutionRetries(0);
-
-		// add consuming topology:
-		Utils.TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema = new Utils.TypeInformationSerializationSchema<Tuple2<Long, byte[]>>(new Tuple2<Long, byte[]>(0L, new byte[]{0}), env.getConfig());
-		Properties consumerProps = new Properties();
-		consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 30));
-		consumerProps.setProperty("max.partition.fetch.bytes", Integer.toString(1024 * 1024 * 30)); // for the new fetcher
-		consumerProps.setProperty("bootstrap.servers", brokerConnectionStrings);
-		consumerProps.setProperty("zookeeper.connect", zookeeperConnectionString);
-		consumerProps.setProperty("group.id", "test");
-		consumerProps.setProperty("auto.commit.enable", "false");
-		consumerProps.setProperty("auto.offset.reset", "earliest");
-
-		FlinkKafkaConsumerBase<Tuple2<Long, byte[]>> source = getConsumer(topic, serSchema, consumerProps);
-		DataStreamSource<Tuple2<Long, byte[]>> consuming = env.addSource(source);
-
-		consuming.addSink(new SinkFunction<Tuple2<Long, byte[]>>() {
-			private static final long serialVersionUID = 1L;
-
-			int elCnt = 0;
-
-			@Override
-			public void invoke(Tuple2<Long, byte[]> value) throws Exception {
-				LOG.info("Received {}", value.f0);
-				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);
-				}
-			}
-		}).setParallelism(1);
-
-		// add producing topology
-		DataStream<Tuple2<Long, byte[]>> stream = env.addSource(new RichSourceFunction<Tuple2<Long, byte[]>>() {
-			private static final long serialVersionUID = 1L;
-			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 {
-				LOG.info("Starting source.");
-				long cnt = 0;
-				Random rnd = new Random(1337);
-				while (running) {
-					//
-					byte[] wl = new byte[Math.abs(rnd.nextInt(1024 * 1024 * 30))];
-					ctx.collect(new Tuple2<Long, byte[]>(cnt++, wl));
-					LOG.info("Emitted cnt=" + (cnt - 1) + " with byte.length = " + wl.length);
-
-					try {
-						Thread.sleep(100);
-					} catch (InterruptedException ignored) {
-					}
-					if(cnt == 10) {
-						LOG.info("Send end signal");
-						// signal end
-						ctx.collect(new Tuple2<Long, byte[]>(-1L, new byte[]{1}));
-						running = false;
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		});
-
-		stream.addSink(new KafkaSink<Tuple2<Long, byte[]>>(brokerConnectionStrings, topic,
-						new Utils.TypeInformationSerializationSchema<Tuple2<Long, byte[]>>(new Tuple2<Long, byte[]>(0L, new byte[]{0}), env.getConfig()))
-		);
-
-		tryExecute(env, "big topology test");
-
-		LOG.info("Finished KafkaITCase.bigRecordTestTopology()");
-	}
-
-
-	@Test
-	public void customPartitioningTestTopology() throws Exception {
-		LOG.info("Starting KafkaITCase.customPartitioningTestTopology()");
-
-		String topic = "customPartitioningTestTopic";
-
-		createTestTopic(topic, 3, 1);
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-		env.setNumberOfExecutionRetries(0);
-
-		Utils.TypeInformationSerializationSchema<Tuple2<Long, String>> serSchema = new Utils.TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig());
-		FlinkKafkaConsumerBase<Tuple2<Long, String>> source = getConsumer(topic, serSchema, standardProps);
-		// add consuming topology:
-		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(source);
-
-		consuming.addSink(new SinkFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-
-			int start = -1;
-			BitSet validator = new BitSet(101);
-
-			boolean gotPartition1 = false;
-			boolean gotPartition2 = false;
-			boolean gotPartition3 = false;
-
-			@Override
-			public void invoke(Tuple2<Long, String> value) throws Exception {
-				LOG.debug("Got " + value);
-				String[] sp = value.f1.split("-");
-				int v = Integer.parseInt(sp[1]);
-
-				assertEquals(value.f0 - 1000, (long) v);
-
-				switch (v) {
-					case 9:
-						gotPartition1 = true;
-						break;
-					case 19:
-						gotPartition2 = true;
-						break;
-					case 99:
-						gotPartition3 = true;
-						break;
-				}
-
-				if (start == -1) {
-					start = v;
-				}
-				Assert.assertFalse("Received tuple twice", validator.get(v - start));
-				validator.set(v - start);
-
-				if (gotPartition1 && gotPartition2 && gotPartition3) {
-					// check if everything in the bitset is set to true
-					int nc;
-					if ((nc = validator.nextClearBit(0)) != 100) {
-						throw new RuntimeException("The bitset was not set to 1 on all elements. Next clear:" + nc + " Set: " + validator);
-					}
-					throw new SuccessException();
-				}
-			}
-		});
-
-		// add producing topology
-		DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Long, String>> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = 0;
-				while (running) {
-					ctx.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++));
-					try {
-						Thread.sleep(100);
-					} catch (InterruptedException ignored) {
-					}
-				}
 			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		});
-		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, new Utils.TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig()), new CustomPartitioner()));
-
-		tryExecute(env, "custom partitioning test");
-
-		LOG.info("Finished KafkaITCase.customPartitioningTestTopology()");
-	}
-
-	/**
-	 * This is for a topic with 3 partitions and Tuple2<Long, String>
-	 */
-	private static class CustomPartitioner implements SerializableKafkaPartitioner {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public int partition(Object key, int numPartitions) {
-
-			@SuppressWarnings("unchecked")
-			Tuple2<Long, String> tuple = (Tuple2<Long, String>) key;
-			if (tuple.f0 < 10) {
-				return 0;
-			} else if (tuple.f0 < 20) {
-				return 1;
-			} else {
-				return 2;
+			catch (Exception e) {
+				LOG.warn("ZK.stop() failed", e);
 			}
+			zookeeper = null;
 		}
-	}
-
-	// ------------------------ Broker failure / exactly once test ---------------------------------
-
-	private static boolean leaderHasShutDown = false;
-	private static boolean shutdownKafkaBroker;
-	final static int NUM_MESSAGES = 200;
-	private static Map<Integer, Integer> finalCount = new HashMap<Integer, Integer>();
-
-	/**
-	 * This test covers:
-	 *  - passing of the execution retries into the JobGraph
-	 *  - Restart behavior of the Kafka Sources in case of a broker failure
-	 * @throws Exception
-	 */
-	@Test(timeout=60000)
-	public void brokerFailureTest() throws Exception {
-		String topic = "brokerFailureTestTopic";
-
-
-		createTestTopic(topic, 2, 2);
-
-		// --------------------------- write data to topic ---------------------
-		LOG.info("Writing data to topic {}", topic);
-
-		Configuration conf = new Configuration();
-		conf.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 s");
-		conf.setString(ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, "4096");
-		conf.setString(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "32");
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(4, conf);
-		env.setParallelism(4);
-		env.getConfig().disableSysoutLogging();
-		env.setNumberOfExecutionRetries(0);
-		env.setBufferTimeout(0);
-
-		DataStreamSource<String> stream = env.addSource(new SourceFunction<String>() {
-			private static final long serialVersionUID = 1L;
-
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<String> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = 0;
-				String payload = "";
-				for(int i = 0; i < 160; i++) {
-					payload += "A";
-				}
-				while (running) {
-					String msg = "kafka-" + (cnt++) + "-"+payload;
-					ctx.collect(msg);
-					if(cnt == NUM_MESSAGES) {
-						LOG.info("Stopping to produce after 200 msgs");
-						break;
-					}
-
-				}
+		
+		// clean up the temp spaces
+		
+		if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpKafkaParent);
 			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got chancel()");
-				running = false;
+			catch (Exception e) {
+				// ignore
 			}
-		});
-		// we write with a parallelism of 1 to ensure that only 200 messages are created
-		stream.setParallelism(1);
-		stream.addSink(new KafkaSink<String>(brokerConnectionStrings, topic, new JavaDefaultStringSchema())).setParallelism(1);
-
-		tryExecute(env, "broker failure test - writer");
-
-		// --------------------------- read and let broker fail ---------------------
-
-		env.setNumberOfExecutionRetries(1); // allow for one restart
-		env.enableCheckpointing(150);
-		LOG.info("Reading data from topic {} and let a broker fail", topic);
-		// find leader to shut down
-		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);
-
-		final String leaderToShutDown = firstPart.leader().get().connectionString();
-		LOG.info("Leader to shutdown {}", leaderToShutDown);
-
-		// add consuming topology:
-
-		FlinkKafkaConsumerBase<String> src = getConsumer(topic, new JavaDefaultStringSchema(), standardProps);
-		DataStreamSource<String> consuming = env.addSource(src);
-		consuming.setParallelism(2).map(new ThrottleMap<String>(10)).setParallelism(2); // read in parallel
-		DataStream<String> mapped = consuming.map(new PassThroughCheckpointed()).setParallelism(4);
-		mapped.addSink(new ExactlyOnceSink(leaderToShutDown)).setParallelism(1); // read only in one instance, to have proper counts
-
-		tryExecute(env, "broker failure test - reader");
-		Assert.assertEquals("Count was not correct", 4, finalCount.size());
-		int count = 0;
-		for(int i = 0; i < 4; i++) {
-			count += finalCount.get(i);
-		}
-		Assert.assertEquals("Count was not correct", NUM_MESSAGES, count); // test for exactly once
-	}
-
-	/**
-	 * Slow down the previous operator by chaining behind it.
-	 * @param <T>
-	 */
-	private static class ThrottleMap<T> implements MapFunction<T,T> {
-
-		int sleep;
-		public ThrottleMap(int sleep) {
-			this.sleep = sleep;
-		}
-
-		@Override
-		public T map(T value) throws Exception {
-			Thread.sleep(this.sleep);
-			return value;
 		}
-	}
-
-	private static class PassThroughCheckpointed extends RichMapFunction<String, String> implements Checkpointed<Integer> {
-
-		private Integer count = 0;
-
-		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			LOG.info("Counter snapshot");
-			return count;
-		}
-
-		@Override
-		public void restoreState(Integer state) {
-			LOG.info("counter restore");
-			this.count = state;
-		}
-
-		@Override
-		public String map(String value) throws Exception {
-			count++;
-			return value;
-		}
-
-		@Override
-		public void close() throws Exception {
-			finalCount.put(getRuntimeContext().getIndexOfThisSubtask(), count);
-		}
-	}
-
-	/**
-	 * A sink ensuring that no data has been send twice through the topology.
-	 * It also fails the one broker (ctor arg) after 15 elements.
-	 */
-	private static class ExactlyOnceSink extends RichSinkFunction<String> implements Checkpointed<ExactlyOnceSink> {
-
-		private static final long serialVersionUID = 1L;
-		int elCnt = 0;
-		BitSet validator = new BitSet(NUM_MESSAGES);
-		private String leaderToShutDown;
-
-		public ExactlyOnceSink(String leaderToShutDown) {
-			this.leaderToShutDown = leaderToShutDown;
-		}
-
-		@Override
-		public void invoke(String value) throws Exception {
-			//LOG.info("Got message = " + value + " leader has shut down " + leaderHasShutDown + " el cnt = " + elCnt);
-			String[] sp = value.split("-");
-			int recordId = Integer.parseInt(sp[1]);
-
-			Assert.assertFalse("Received tuple with value " + recordId + " twice", validator.get(recordId));
-			validator.set(recordId);
-
-			elCnt++;
-			if (elCnt == 15 && !shutdownKafkaBroker) {
-				// shut down a Kafka broker
-				shutdownKafkaBroker = true;
-				LOG.info("Stopping lead broker");
-				for (KafkaServer kafkaServer : brokers) {
-					if (leaderToShutDown.equals(kafkaServer.config().advertisedHostName()+ ":"+ kafkaServer.config().advertisedPort())) {
-						LOG.info("Stopping broker {}", leaderToShutDown);
-						kafkaServer.shutdown();
-						leaderHasShutDown = true;
-						break;
-					}
-				}
-				Assert.assertTrue("unable to find leader", leaderHasShutDown);
+		if (tmpZkDir != null && tmpZkDir.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpZkDir);
 			}
-
-			if (leaderHasShutDown) { // it only makes sense to check once the shutdown is completed
-				if (elCnt >= NUM_MESSAGES) {
-					// check if everything in the bitset is set to true
-					int nc;
-					if ((nc = validator.nextClearBit(0)) != NUM_MESSAGES) {
-						throw new RuntimeException("The bitset was not set to 1 on all elements to be checked. Next clear:" + nc + " Set: " + validator);
-					}
-					throw new SuccessException();
-				}
+			catch (Exception e) {
+				// ignore
 			}
 		}
 
-		@Override
-		public ExactlyOnceSink snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			LOG.info("Snapshotting state of sink");
-			return this;
-		}
-
-		@Override
-		public void restoreState(ExactlyOnceSink state) {
-			LOG.info("Restoring state of sink");
-			this.elCnt = state.elCnt;
-			this.validator = state.validator;
-		}
-	}
-
-	// -------------------------------- Utilities --------------------------------------------------
-
-
-	/**
-	 * Execute stream.
-	 * forwards any exception except the SuccessException
-	 *
-	 */
-	public static void tryExecute(StreamExecutionEnvironment see, String name) throws Exception {
-		try {
-			see.execute(name);
-		} catch (JobExecutionException good) {
-			Throwable t = good.getCause();
-			int limit = 0;
-			while (!(t instanceof SuccessException)) {
-				if(t == null) {
-					LOG.warn("Test failed with exception", good);
-					Assert.fail("Test failed with: " + good.getMessage());
-				}
-
-				t = t.getCause();
-				if (limit++ == 20) {
-					LOG.warn("Test failed with exception", good);
-					Assert.fail("Test failed with: " + good.getMessage());
-				}
-			}
-		}
-	}
-
-	private void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
-		// create topic
-		Properties topicConfig = new Properties();
-		LOG.info("Creating topic {}", topic);
-		AdminUtils.createTopic(zkClient, topic, numberOfPartitions, replicationFactor, topicConfig);
-		try {
-			// I know, bad style.
-			Thread.sleep(500);
-		} catch (InterruptedException e) {
-		}
-	}
-
-	private static TestingServer getZookeeper() throws Exception {
-		return new TestingServer(zkPort, tmpZkDir);
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    KafkaITCase finished"); 
+		LOG.info("-------------------------------------------------------------------------");
 	}
 
 	/**
 	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
 	 */
-	private static KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws UnknownHostException {
+	private static KafkaServer getKafkaServer(int brokerId, File tmpFolder,
+												String kafkaHost,
+												String zookeeperConnectionString) throws Exception {
 		Properties kafkaProperties = new Properties();
 
 		int kafkaPort = NetUtils.getAvailablePort();
@@ -1062,8 +248,8 @@ public abstract class KafkaTestBase {
 		kafkaProperties.put("broker.id", Integer.toString(brokerId));
 		kafkaProperties.put("log.dir", tmpFolder.toString());
 		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
-		kafkaProperties.put("message.max.bytes", "" + (35 * 1024 * 1024));
-		kafkaProperties.put("replica.fetch.max.bytes", "" + (35 * 1024 * 1024));
+		kafkaProperties.put("message.max.bytes", "" + (50 * 1024 * 1024));
+		kafkaProperties.put("replica.fetch.max.bytes", "" + (50 * 1024 * 1024));
 		KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
 
 		KafkaServer server = new KafkaServer(kafkaConfig, new KafkaLocalSystemTime());
@@ -1071,158 +257,73 @@ public abstract class KafkaTestBase {
 		return server;
 	}
 
-	public static class SuccessException extends Exception {
-		private static final long serialVersionUID = 1L;
+	// ------------------------------------------------------------------------
+	//  Execution utilities
+	// ------------------------------------------------------------------------
+	
+	protected ZkClient createZookeeperClient() {
+		return new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
+				standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
 	}
-
-	private static class MockRuntimeContext implements RuntimeContext {
-
-		int numberOfParallelSubtasks = 0;
-		int indexOfThisSubtask = 0;
-		@Override
-		public String getTaskName() {
-			return null;
-		}
-
-		@Override
-		public int getNumberOfParallelSubtasks() {
-			return numberOfParallelSubtasks;
-		}
-
-		@Override
-		public int getIndexOfThisSubtask() {
-			return indexOfThisSubtask;
-		}
-
-		@Override
-		public ExecutionConfig getExecutionConfig() {
-			return null;
-		}
-
-		@Override
-		public ClassLoader getUserCodeClassLoader() {
-			return null;
-		}
-
-		@Override
-		public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator) {
-
-		}
-
-		@Override
-		public <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name) {
-			return null;
-		}
-
-		@Override
-		public Map<String, Accumulator<?, ?>> getAllAccumulators() {
-			return null;
-		}
-
-		@Override
-		public IntCounter getIntCounter(String name) {
-			return null;
-		}
-
-		@Override
-		public LongCounter getLongCounter(String name) {
-			return null;
-		}
-
-		@Override
-		public DoubleCounter getDoubleCounter(String name) {
-			return null;
-		}
-
-		@Override
-		public Histogram getHistogram(String name) {
-			return null;
-		}
-
-		@Override
-		public <RT> List<RT> getBroadcastVariable(String name) {
-			return null;
-		}
-
-		@Override
-		public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
-			return null;
-		}
-
-		@Override
-		public DistributedCache getDistributedCache() {
-			return null;
-		}
-
-		@Override
-		public <S, C extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState, boolean partitioned, StateCheckpointer<S, C> checkpointer) throws IOException {
-			return null;
+	
+	protected static void tryExecute(StreamExecutionEnvironment see, String name) throws Exception {
+		try {
+			see.execute(name);
 		}
-
-		@Override
-		public <S extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState, boolean partitioned) throws IOException {
-			return null;
+		catch (ProgramInvocationException | JobExecutionException root) {
+			Throwable cause = root.getCause();
+			
+			// search for nested SuccessExceptions
+			int depth = 0;
+			while (!(cause instanceof SuccessException)) {
+				if (cause == null || depth++ == 20) {
+					root.printStackTrace();
+					fail("Test failed: " + root.getMessage());
+				}
+				else {
+					cause = cause.getCause();
+				}
+			}
 		}
 	}
 
-	// ----------------------- Debugging utilities --------------------
-
-	/**
-	 * Read topic to list, only using Kafka code.
-	 * @return
-	 */
-	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();
+	protected static void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
+		
+		// create topic with one client
+		Properties topicConfig = new Properties();
+		LOG.info("Creating topic {}", topic);
 
-		List<MessageAndMetadata<byte[], byte[]>> result = new ArrayList<MessageAndMetadata<byte[], byte[]>>();
-		int read = 0;
-		while(iteratorToRead.hasNext()) {
-			read++;
-			result.add(iteratorToRead.next());
-			if(read == stopAfter) {
-				LOG.info("Read "+read+" elements");
-				return result;
+		ZkClient creator = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
+				standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
+		
+		AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig);
+		creator.close();
+		
+		// validate that the topic has been created
+
+		try (KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(
+				standardProps, null, new ByteArrayDeserializer(), new ByteArrayDeserializer()))
+		{
+			final long deadline = System.currentTimeMillis() + 30000;
+			do {
+				List<PartitionInfo> partitions = consumer.partitionsFor(topic);
+				if (partitions != null && partitions.size() > 0) {
+					return;
+				}
 			}
+			while (System.currentTimeMillis() < deadline);
+			fail("Test topic could not be created");
 		}
-		return result;
 	}
+	
+	protected static void deleteTestTopic(String topic) {
+		LOG.info("Deleting topic {}", topic);
 
-	private static void printTopic(String topicName, ConsumerConfig config, DeserializationSchema deserializationSchema, int stopAfter){
-		List<MessageAndMetadata<byte[], byte[]>> contents = readTopicToList(topicName, config, stopAfter);
-		LOG.info("Printing contents of topic {} in consumer grouo {}", topicName, config.groupId());
-		for(MessageAndMetadata<byte[], byte[]> message: contents) {
-			Object out = deserializationSchema.deserialize(message.message());
-			LOG.info("Message: partition: {} offset: {} msg: {}", message.partition(), message.offset(), out.toString());
-		}
-	}
-
-	private static void printTopic(String topicName, int elements, ExecutionConfig ec) {
-		// write the sequence to log for debugging purposes
-		Properties stdProps = standardCC.props().props();
-		Properties newProps = new Properties(stdProps);
-		newProps.setProperty("group.id", "topic-printer"+UUID.randomUUID().toString());
-		newProps.setProperty("auto.offset.reset", "smallest");
-		newProps.setProperty("zookeeper.connect", standardCC.zkConnect());
+		ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
+				standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
 
-		ConsumerConfig printerConfig = new ConsumerConfig(newProps);
-		DeserializationSchema deserializer = new Utils.TypeInformationSerializationSchema<Tuple2<Integer, Integer>>(new Tuple2<Integer, Integer>(1,1), ec);
-		printTopic(topicName, printerConfig, deserializer, elements);
+		AdminUtils.deleteTopic(zk, topic);
+		
+		zk.close();
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.java
new file mode 100644
index 0000000..c412136
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/internals/ZookeeperOffsetHandlerTest.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.internals;
+
+import kafka.admin.AdminUtils;
+
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.flink.streaming.connectors.KafkaTestBase;
+
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class ZookeeperOffsetHandlerTest extends KafkaTestBase {
+	
+	@Test
+	public void runOffsetManipulationinZooKeeperTest() {
+		try {
+			final String topicName = "ZookeeperOffsetHandlerTest-Topic";
+			final String groupId = "ZookeeperOffsetHandlerTest-Group";
+			
+			final long offset = (long) (Math.random() * Long.MAX_VALUE);
+
+			ZkClient zkClient = createZookeeperClient();
+			AdminUtils.createTopic(zkClient, topicName, 3, 2, new Properties());
+				
+			ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, groupId, topicName, 0, offset);
+	
+			long fetchedOffset = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, groupId, topicName, 0);
+
+			zkClient.close();
+			
+			assertEquals(offset, fetchedOffset);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/MockRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/MockRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/MockRuntimeContext.java
new file mode 100644
index 0000000..1f71271
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/MockRuntimeContext.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.streaming.connectors.testutils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.accumulators.DoubleCounter;
+import org.apache.flink.api.common.accumulators.Histogram;
+import org.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.cache.DistributedCache;
+import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.OperatorState;
+import org.apache.flink.api.common.state.StateCheckpointer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+public class MockRuntimeContext implements RuntimeContext {
+
+	private final int numberOfParallelSubtasks;
+	private final int indexOfThisSubtask;
+
+	public MockRuntimeContext(int numberOfParallelSubtasks, int indexOfThisSubtask) {
+		this.numberOfParallelSubtasks = numberOfParallelSubtasks;
+		this.indexOfThisSubtask = indexOfThisSubtask;
+	}
+
+
+	@Override
+	public String getTaskName() {
+		return null;
+	}
+
+	@Override
+	public int getNumberOfParallelSubtasks() {
+		return numberOfParallelSubtasks;
+	}
+
+	@Override
+	public int getIndexOfThisSubtask() {
+		return indexOfThisSubtask;
+	}
+
+	@Override
+	public ExecutionConfig getExecutionConfig() {
+		return null;
+	}
+
+	@Override
+	public ClassLoader getUserCodeClassLoader() {
+		return null;
+	}
+
+	@Override
+	public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator) {}
+
+	@Override
+	public <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name) {
+		return null;
+	}
+
+	@Override
+	public Map<String, Accumulator<?, ?>> getAllAccumulators() {
+		return null;
+	}
+
+	@Override
+	public IntCounter getIntCounter(String name) {
+		return null;
+	}
+
+	@Override
+	public LongCounter getLongCounter(String name) {
+		return null;
+	}
+
+	@Override
+	public DoubleCounter getDoubleCounter(String name) {
+		return null;
+	}
+
+	@Override
+	public Histogram getHistogram(String name) {
+		return null;
+	}
+
+	@Override
+	public <RT> List<RT> getBroadcastVariable(String name) {
+		return null;
+	}
+
+	@Override
+	public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
+		return null;
+	}
+
+	@Override
+	public DistributedCache getDistributedCache() {
+		return null;
+	}
+
+	@Override
+	public <S, C extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState, boolean partitioned, StateCheckpointer<S, C> checkpointer) throws IOException {
+		return null;
+	}
+
+	@Override
+	public <S extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState, boolean partitioned) throws IOException {
+		return null;
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ThrottledMapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ThrottledMapper.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ThrottledMapper.java
new file mode 100644
index 0000000..872d42f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/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.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/b9892a0e/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.java
new file mode 100644
index 0000000..1e5f027
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.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.testutils;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.connectors.SerializableKafkaPartitioner;
+
+/**
+ * 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 implements SerializableKafkaPartitioner {
+	
+	private static final long serialVersionUID = 1L;
+
+	private final int expectedPartitions;
+
+	
+	public Tuple2Partitioner(int expectedPartitions) {
+		this.expectedPartitions = expectedPartitions;
+	}
+
+	@Override
+	public int partition(Object key, int numPartitions) {
+		if (numPartitions != expectedPartitions) {
+			throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions");
+		}
+		@SuppressWarnings("unchecked")
+		Tuple2<Integer, Integer> element = (Tuple2<Integer, Integer>) key;
+		
+		return element.f0;
+	}
+}
\ No newline at end of file


[02/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/MetadataRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/MetadataRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/MetadataRequest.java
deleted file mode 100644
index a42333e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/MetadataRequest.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.Cluster;
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.PartitionInfo;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class MetadataRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id);
-    private static final String TOPICS_KEY_NAME = "topics";
-
-    private final List<String> topics;
-
-    public MetadataRequest(List<String> topics) {
-        super(new Struct(CURRENT_SCHEMA));
-        struct.set(TOPICS_KEY_NAME, topics.toArray());
-        this.topics = topics;
-    }
-
-    public MetadataRequest(Struct struct) {
-        super(struct);
-        Object[] topicArray = struct.getArray(TOPICS_KEY_NAME);
-        topics = new ArrayList<String>();
-        for (Object topicObj: topicArray) {
-            topics.add((String) topicObj);
-        }
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<String, Errors> topicErrors = new HashMap<String, Errors>();
-        for (String topic : topics) {
-            topicErrors.put(topic, Errors.forException(e));
-        }
-
-        Cluster cluster = new Cluster(new ArrayList<Node>(), new ArrayList<PartitionInfo>());
-        switch (versionId) {
-            case 0:
-                return new MetadataResponse(cluster, topicErrors);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id)));
-        }
-    }
-
-    public List<String> topics() {
-        return topics;
-    }
-
-    public static MetadataRequest parse(ByteBuffer buffer, int versionId) {
-        return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer));
-    }
-
-    public static MetadataRequest parse(ByteBuffer buffer) {
-        return new MetadataRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/MetadataResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/MetadataResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/MetadataResponse.java
deleted file mode 100644
index b7db4fa..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/MetadataResponse.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.Cluster;
-import org.apache.kafka.copied.common.Node;
-import org.apache.kafka.copied.common.PartitionInfo;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class MetadataResponse extends AbstractRequestResponse {
-
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id);
-    private static final String BROKERS_KEY_NAME = "brokers";
-    private static final String TOPIC_METATDATA_KEY_NAME = "topic_metadata";
-
-    // broker level field names
-    private static final String NODE_ID_KEY_NAME = "node_id";
-    private static final String HOST_KEY_NAME = "host";
-    private static final String PORT_KEY_NAME = "port";
-
-    // topic level field names
-    private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code";
-
-    /**
-     * Possible error code:
-     *
-     * TODO
-     */
-
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata";
-
-    // partition level field names
-    private static final String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code";
-
-    /**
-     * Possible error code:
-     *
-     * TODO
-     */
-
-    private static final String PARTITION_KEY_NAME = "partition_id";
-    private static final String LEADER_KEY_NAME = "leader";
-    private static final String REPLICAS_KEY_NAME = "replicas";
-    private static final String ISR_KEY_NAME = "isr";
-
-    private final Cluster cluster;
-    private final Map<String, Errors> errors;
-
-    /**
-     * Constructor for MetadataResponse where there are errors for some of the topics,
-     * error data take precedence over cluster information for particular topic
-     */
-    public MetadataResponse(Cluster cluster, Map<String, Errors> errors) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        List<Struct> brokerArray = new ArrayList<Struct>();
-        for (Node node : cluster.nodes()) {
-            Struct broker = struct.instance(BROKERS_KEY_NAME);
-            broker.set(NODE_ID_KEY_NAME, node.id());
-            broker.set(HOST_KEY_NAME, node.host());
-            broker.set(PORT_KEY_NAME, node.port());
-            brokerArray.add(broker);
-        }
-        struct.set(BROKERS_KEY_NAME, brokerArray.toArray());
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (String topic : cluster.topics()) {
-            Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME);
-
-            topicData.set(TOPIC_KEY_NAME, topic);
-            if (errors.containsKey(topic)) {
-                topicData.set(TOPIC_ERROR_CODE_KEY_NAME, errors.get(topic).code());
-            } else {
-                topicData.set(TOPIC_ERROR_CODE_KEY_NAME, Errors.NONE.code());
-                List<Struct> partitionArray = new ArrayList<Struct>();
-                for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) {
-                    Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME);
-                    partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, Errors.NONE.code());
-                    partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition());
-                    partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id());
-                    ArrayList<Integer> replicas = new ArrayList<Integer>();
-                    for (Node node : fetchPartitionData.replicas())
-                        replicas.add(node.id());
-                    partitionData.set(REPLICAS_KEY_NAME, replicas.toArray());
-                    ArrayList<Integer> isr = new ArrayList<Integer>();
-                    for (Node node : fetchPartitionData.inSyncReplicas())
-                        isr.add(node.id());
-                    partitionData.set(ISR_KEY_NAME, isr.toArray());
-                    partitionArray.add(partitionData);
-                }
-                topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray());
-            }
-
-            topicArray.add(topicData);
-        }
-        struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray());
-
-        this.cluster = cluster;
-        this.errors = new HashMap<String, Errors>();
-    }
-
-    public MetadataResponse(Struct struct) {
-        super(struct);
-        Map<String, Errors> errors = new HashMap<String, Errors>();
-        Map<Integer, Node> brokers = new HashMap<Integer, Node>();
-        Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME);
-        for (int i = 0; i < brokerStructs.length; i++) {
-            Struct broker = (Struct) brokerStructs[i];
-            int nodeId = broker.getInt(NODE_ID_KEY_NAME);
-            String host = broker.getString(HOST_KEY_NAME);
-            int port = broker.getInt(PORT_KEY_NAME);
-            brokers.put(nodeId, new Node(nodeId, host, port));
-        }
-        List<PartitionInfo> partitions = new ArrayList<PartitionInfo>();
-        Object[] topicInfos = (Object[]) struct.get(TOPIC_METATDATA_KEY_NAME);
-        for (int i = 0; i < topicInfos.length; i++) {
-            Struct topicInfo = (Struct) topicInfos[i];
-            short topicError = topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME);
-            String topic = topicInfo.getString(TOPIC_KEY_NAME);
-            if (topicError == Errors.NONE.code()) {
-                Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME);
-                for (int j = 0; j < partitionInfos.length; j++) {
-                    Struct partitionInfo = (Struct) partitionInfos[j];
-                    int partition = partitionInfo.getInt(PARTITION_KEY_NAME);
-                    int leader = partitionInfo.getInt(LEADER_KEY_NAME);
-                    Node leaderNode = leader == -1 ? null : brokers.get(leader);
-                    Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME);
-                    Node[] replicaNodes = new Node[replicas.length];
-                    for (int k = 0; k < replicas.length; k++)
-                        replicaNodes[k] = brokers.get(replicas[k]);
-                    Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME);
-                    Node[] isrNodes = new Node[isr.length];
-                    for (int k = 0; k < isr.length; k++)
-                        isrNodes[k] = brokers.get(isr[k]);
-                    partitions.add(new PartitionInfo(topic, partition, leaderNode, replicaNodes, isrNodes));
-                }
-            } else {
-                errors.put(topic, Errors.forCode(topicError));
-            }
-        }
-        this.errors = errors;
-        this.cluster = new Cluster(brokers.values(), partitions);
-    }
-
-    public Map<String, Errors> errors() {
-        return this.errors;
-    }
-
-    public Cluster cluster() {
-        return this.cluster;
-    }
-
-    public static MetadataResponse parse(ByteBuffer buffer) {
-        return new MetadataResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetCommitRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetCommitRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetCommitRequest.java
deleted file mode 100644
index 20e518f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetCommitRequest.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * This wrapper supports both v0 and v1 of OffsetCommitRequest.
- */
-public class OffsetCommitRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-    private static final String GENERATION_ID_KEY_NAME = "group_generation_id";
-    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String RETENTION_TIME_KEY_NAME = "retention_time";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
-    private static final String METADATA_KEY_NAME = "metadata";
-
-    @Deprecated
-    private static final String TIMESTAMP_KEY_NAME = "timestamp";         // for v0, v1
-
-    // default values for the current version
-    public static final int DEFAULT_GENERATION_ID = -1;
-    public static final String DEFAULT_CONSUMER_ID = "";
-    public static final long DEFAULT_RETENTION_TIME = -1L;
-
-    // default values for old versions,
-    // will be removed after these versions are deprecated
-    @Deprecated
-    public static final long DEFAULT_TIMESTAMP = -1L;            // for V0, V1
-
-    private final String groupId;
-    private final String consumerId;
-    private final int generationId;
-    private final long retentionTime;
-    private final Map<TopicPartition, PartitionData> offsetData;
-
-    public static final class PartitionData {
-        @Deprecated
-        public final long timestamp;                // for V1
-
-        public final long offset;
-        public final String metadata;
-
-        @Deprecated
-        public PartitionData(long offset, long timestamp, String metadata) {
-            this.offset = offset;
-            this.timestamp = timestamp;
-            this.metadata = metadata;
-        }
-
-        public PartitionData(long offset, String metadata) {
-            this(offset, DEFAULT_TIMESTAMP, metadata);
-        }
-    }
-
-    /**
-     * Constructor for version 0.
-     * @param groupId
-     * @param offsetData
-     */
-    @Deprecated
-    public OffsetCommitRequest(String groupId, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0)));
-
-        initCommonFields(groupId, offsetData);
-        this.groupId = groupId;
-        this.generationId = DEFAULT_GENERATION_ID;
-        this.consumerId = DEFAULT_CONSUMER_ID;
-        this.retentionTime = DEFAULT_RETENTION_TIME;
-        this.offsetData = offsetData;
-    }
-
-    /**
-     * Constructor for version 1.
-     * @param groupId
-     * @param generationId
-     * @param consumerId
-     * @param offsetData
-     */
-    @Deprecated
-    public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1)));
-
-        initCommonFields(groupId, offsetData);
-        struct.set(GENERATION_ID_KEY_NAME, generationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        this.groupId = groupId;
-        this.generationId = generationId;
-        this.consumerId = consumerId;
-        this.retentionTime = DEFAULT_RETENTION_TIME;
-        this.offsetData = offsetData;
-    }
-
-    /**
-     * Constructor for version 2.
-     * @param groupId
-     * @param generationId
-     * @param consumerId
-     * @param retentionTime
-     * @param offsetData
-     */
-    public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        initCommonFields(groupId, offsetData);
-        struct.set(GENERATION_ID_KEY_NAME, generationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        struct.set(RETENTION_TIME_KEY_NAME, retentionTime);
-        this.groupId = groupId;
-        this.generationId = generationId;
-        this.consumerId = consumerId;
-        this.retentionTime = retentionTime;
-        this.offsetData = offsetData;
-    }
-
-    private void initCommonFields(String groupId, Map<TopicPartition, PartitionData> offsetData) {
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
-
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        List<Struct> topicArray = new ArrayList<Struct>();
-
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
-                PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
-                // Only for v1
-                if (partitionData.hasField(TIMESTAMP_KEY_NAME))
-                    partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp);
-                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
-    }
-
-    public OffsetCommitRequest(Struct struct) {
-        super(struct);
-
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-        // This field only exists in v1.
-        if (struct.hasField(GENERATION_ID_KEY_NAME))
-            generationId = struct.getInt(GENERATION_ID_KEY_NAME);
-        else
-            generationId = DEFAULT_GENERATION_ID;
-
-        // This field only exists in v1.
-        if (struct.hasField(CONSUMER_ID_KEY_NAME))
-            consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
-        else
-            consumerId = DEFAULT_CONSUMER_ID;
-
-        // This field only exists in v2
-        if (struct.hasField(RETENTION_TIME_KEY_NAME))
-            retentionTime = struct.getLong(RETENTION_TIME_KEY_NAME);
-        else
-            retentionTime = DEFAULT_RETENTION_TIME;
-
-        offsetData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicDataObj : struct.getArray(TOPICS_KEY_NAME)) {
-            Struct topicData = (Struct) topicDataObj;
-            String topic = topicData.getString(TOPIC_KEY_NAME);
-            for (Object partitionDataObj : topicData.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionDataStruct = (Struct) partitionDataObj;
-                int partition = partitionDataStruct.getInt(PARTITION_KEY_NAME);
-                long offset = partitionDataStruct.getLong(COMMIT_OFFSET_KEY_NAME);
-                String metadata = partitionDataStruct.getString(METADATA_KEY_NAME);
-                PartitionData partitionOffset;
-                // This field only exists in v1
-                if (partitionDataStruct.hasField(TIMESTAMP_KEY_NAME)) {
-                    long timestamp = partitionDataStruct.getLong(TIMESTAMP_KEY_NAME);
-                    partitionOffset = new PartitionData(offset, timestamp, metadata);
-                } else {
-                    partitionOffset = new PartitionData(offset, metadata);
-                }
-                offsetData.put(new TopicPartition(topic, partition), partitionOffset);
-            }
-        }
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<TopicPartition, Short> responseData = new HashMap<TopicPartition, Short>();
-        for (Map.Entry<TopicPartition, PartitionData> entry: offsetData.entrySet()) {
-            responseData.put(entry.getKey(), Errors.forException(e).code());
-        }
-
-        switch (versionId) {
-            // OffsetCommitResponseV0 == OffsetCommitResponseV1 == OffsetCommitResponseV2
-            case 0:
-            case 1:
-            case 2:
-                return new OffsetCommitResponse(responseData);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_COMMIT.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public int generationId() {
-        return generationId;
-    }
-
-    public String consumerId() {
-        return consumerId;
-    }
-
-    public long retentionTime() {
-        return retentionTime;
-    }
-
-    public Map<TopicPartition, PartitionData> offsetData() {
-        return offsetData;
-    }
-
-    public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) {
-        Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId);
-        return new OffsetCommitRequest((Struct) schema.read(buffer));
-    }
-
-    public static OffsetCommitRequest parse(ByteBuffer buffer) {
-        return new OffsetCommitRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetCommitResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetCommitResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetCommitResponse.java
deleted file mode 100644
index 59f621e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetCommitResponse.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class OffsetCommitResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level fields
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    // partition level fields
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    /**
-     * Possible error code:
-     *
-     * OFFSET_METADATA_TOO_LARGE (12)
-     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR_FOR_CONSUMER (16)
-     * ILLEGAL_GENERATION (22)
-     * UNKNOWN_CONSUMER_ID (25)
-     * COMMITTING_PARTITIONS_NOT_ASSIGNED (27)
-     * INVALID_COMMIT_OFFSET_SIZE (28)
-     */
-
-    private final Map<TopicPartition, Short> responseData;
-
-    public OffsetCommitResponse(Map<TopicPartition, Short> responseData) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        Map<String, Map<Integer, Short>> topicsData = CollectionUtils.groupDataByTopic(responseData);
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, Short>> entries: topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, Short> partitionEntry : entries.getValue().entrySet()) {
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(ERROR_CODE_KEY_NAME, partitionEntry.getValue());
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
-        this.responseData = responseData;
-    }
-
-    public OffsetCommitResponse(Struct struct) {
-        super(struct);
-        responseData = new HashMap<TopicPartition, Short>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
-                responseData.put(new TopicPartition(topic, partition), errorCode);
-            }
-        }
-    }
-
-    public Map<TopicPartition, Short> responseData() {
-        return responseData;
-    }
-
-    public static OffsetCommitResponse parse(ByteBuffer buffer) {
-        return new OffsetCommitResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetFetchRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetFetchRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetFetchRequest.java
deleted file mode 100644
index 242d491..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetFetchRequest.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * This wrapper supports both v0 and v1 of OffsetFetchRequest.
- */
-public class OffsetFetchRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-    private static final String TOPICS_KEY_NAME = "topics";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-
-    private final String groupId;
-    private final List<TopicPartition> partitions;
-
-    public OffsetFetchRequest(String groupId, List<TopicPartition> partitions) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        Map<String, List<Integer>> topicsData = CollectionUtils.groupDataByTopic(partitions);
-
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, List<Integer>> entries: topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Integer partiitonId : entries.getValue()) {
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partiitonId);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
-        this.groupId = groupId;
-        this.partitions = partitions;
-    }
-
-    public OffsetFetchRequest(Struct struct) {
-        super(struct);
-        partitions = new ArrayList<TopicPartition>();
-        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                partitions.add(new TopicPartition(topic, partition));
-            }
-        }
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = new HashMap<TopicPartition, OffsetFetchResponse.PartitionData>();
-
-        for (TopicPartition partition: partitions) {
-            responseData.put(partition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET,
-                    OffsetFetchResponse.NO_METADATA,
-                    Errors.forException(e).code()));
-        }
-
-        switch (versionId) {
-            // OffsetFetchResponseV0 == OffsetFetchResponseV1
-            case 0:
-            case 1:
-                return new OffsetFetchResponse(responseData);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public List<TopicPartition> partitions() {
-        return partitions;
-    }
-
-    public static OffsetFetchRequest parse(ByteBuffer buffer, int versionId) {
-        return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer));
-    }
-
-    public static OffsetFetchRequest parse(ByteBuffer buffer) {
-        return new OffsetFetchRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetFetchResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetFetchResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetFetchResponse.java
deleted file mode 100644
index 8a214a8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/OffsetFetchResponse.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class OffsetFetchResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level fields
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    // partition level fields
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
-    private static final String METADATA_KEY_NAME = "metadata";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    public static final long INVALID_OFFSET = -1L;
-    public static final String NO_METADATA = "";
-
-    /**
-     * Possible error code:
-     *
-     *  UNKNOWN_TOPIC_OR_PARTITION (3)  <- only for request v0
-     *  OFFSET_LOAD_IN_PROGRESS (14)
-     *  NOT_COORDINATOR_FOR_CONSUMER (16)
-     *  ILLEGAL_GENERATION (22)
-     *  UNKNOWN_CONSUMER_ID (25)
-     */
-
-    private final Map<TopicPartition, PartitionData> responseData;
-
-    public static final class PartitionData {
-        public final long offset;
-        public final String metadata;
-        public final short errorCode;
-
-        public PartitionData(long offset, String metadata, short errorCode) {
-            this.offset = offset;
-            this.metadata = metadata;
-            this.errorCode = errorCode;
-        }
-
-        public boolean hasError() {
-            return this.errorCode != Errors.NONE.code();
-        }
-    }
-
-    public OffsetFetchResponse(Map<TopicPartition, PartitionData> responseData) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> entries : topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : entries.getValue().entrySet()) {
-                PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
-                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
-                partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
-        this.responseData = responseData;
-    }
-
-    public OffsetFetchResponse(Struct struct) {
-        super(struct);
-        responseData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME);
-                String metadata = partitionResponse.getString(METADATA_KEY_NAME);
-                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
-                PartitionData partitionData = new PartitionData(offset, metadata, errorCode);
-                responseData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    public Map<TopicPartition, PartitionData> responseData() {
-        return responseData;
-    }
-
-    public static OffsetFetchResponse parse(ByteBuffer buffer) {
-        return new OffsetFetchResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ProduceRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ProduceRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ProduceRequest.java
deleted file mode 100644
index 7a1c995..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ProduceRequest.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-
-package org.apache.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.Errors;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class ProduceRequest  extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id);
-    private static final String ACKS_KEY_NAME = "acks";
-    private static final String TIMEOUT_KEY_NAME = "timeout";
-    private static final String TOPIC_DATA_KEY_NAME = "topic_data";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_DATA_KEY_NAME = "data";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String RECORD_SET_KEY_NAME = "record_set";
-
-    private final short acks;
-    private final int timeout;
-    private final Map<TopicPartition, ByteBuffer> partitionRecords;
-
-    public ProduceRequest(short acks, int timeout, Map<TopicPartition, ByteBuffer> partitionRecords) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, ByteBuffer>> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords);
-        struct.set(ACKS_KEY_NAME, acks);
-        struct.set(TIMEOUT_KEY_NAME, timeout);
-        List<Struct> topicDatas = new ArrayList<Struct>(recordsByTopic.size());
-        for (Map.Entry<String, Map<Integer, ByteBuffer>> entry : recordsByTopic.entrySet()) {
-            Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, ByteBuffer> partitionEntry : entry.getValue().entrySet()) {
-                ByteBuffer buffer = partitionEntry.getValue().duplicate();
-                Struct part = topicData.instance(PARTITION_DATA_KEY_NAME)
-                                       .set(PARTITION_KEY_NAME, partitionEntry.getKey())
-                                       .set(RECORD_SET_KEY_NAME, buffer);
-                partitionArray.add(part);
-            }
-            topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray());
-            topicDatas.add(topicData);
-        }
-        struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray());
-        this.acks = acks;
-        this.timeout = timeout;
-        this.partitionRecords = partitionRecords;
-    }
-
-    public ProduceRequest(Struct struct) {
-        super(struct);
-        partitionRecords = new HashMap<TopicPartition, ByteBuffer>();
-        for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) {
-            Struct topicData = (Struct) topicDataObj;
-            String topic = topicData.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicData.getArray(PARTITION_DATA_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                ByteBuffer records = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
-                partitionRecords.put(new TopicPartition(topic, partition), records);
-            }
-        }
-        acks = struct.getShort(ACKS_KEY_NAME);
-        timeout = struct.getInt(TIMEOUT_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        /* In case the producer doesn't actually want any response */
-        if (acks == 0)
-            return null;
-
-        Map<TopicPartition, ProduceResponse.PartitionResponse> responseMap = new HashMap<TopicPartition, ProduceResponse.PartitionResponse>();
-
-        for (Map.Entry<TopicPartition, ByteBuffer> entry : partitionRecords.entrySet()) {
-            responseMap.put(entry.getKey(), new ProduceResponse.PartitionResponse(Errors.forException(e).code(), ProduceResponse.INVALID_OFFSET));
-        }
-
-        switch (versionId) {
-            case 0:
-                return new ProduceResponse(responseMap);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)));
-        }
-    }
-
-    public short acks() {
-        return acks;
-    }
-
-    public int timeout() {
-        return timeout;
-    }
-
-    public Map<TopicPartition, ByteBuffer> partitionRecords() {
-        return partitionRecords;
-    }
-
-    public static ProduceRequest parse(ByteBuffer buffer, int versionId) {
-        return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer));
-    }
-
-    public static ProduceRequest parse(ByteBuffer buffer) {
-        return new ProduceRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ProduceResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ProduceResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ProduceResponse.java
deleted file mode 100644
index 47313ad..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ProduceResponse.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.TopicPartition;
-import org.apache.kafka.copied.common.protocol.ApiKeys;
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.types.Schema;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-import org.apache.kafka.copied.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class ProduceResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    public static final long INVALID_OFFSET = -1L;
-
-    /**
-     * Possible error code:
-     *
-     * TODO
-     */
-
-    private static final String BASE_OFFSET_KEY_NAME = "base_offset";
-
-    private final Map<TopicPartition, PartitionResponse> responses;
-
-    public ProduceResponse(Map<TopicPartition, PartitionResponse> responses) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionResponse>> responseByTopic = CollectionUtils.groupDataByTopic(responses);
-        List<Struct> topicDatas = new ArrayList<Struct>(responseByTopic.size());
-        for (Map.Entry<String, Map<Integer, PartitionResponse>> entry : responseByTopic.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionResponse> partitionEntry : entry.getValue().entrySet()) {
-                PartitionResponse part = partitionEntry.getValue();
-                Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME)
-                                       .set(PARTITION_KEY_NAME, partitionEntry.getKey())
-                                       .set(ERROR_CODE_KEY_NAME, part.errorCode)
-                                       .set(BASE_OFFSET_KEY_NAME, part.baseOffset);
-                partitionArray.add(partStruct);
-            }
-            topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray());
-            topicDatas.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicDatas.toArray());
-        this.responses = responses;
-    }
-
-    public ProduceResponse(Struct struct) {
-        super(struct);
-        responses = new HashMap<TopicPartition, PartitionResponse>();
-        for (Object topicResponse : struct.getArray("responses")) {
-            Struct topicRespStruct = (Struct) topicResponse;
-            String topic = topicRespStruct.getString("topic");
-            for (Object partResponse : topicRespStruct.getArray("partition_responses")) {
-                Struct partRespStruct = (Struct) partResponse;
-                int partition = partRespStruct.getInt("partition");
-                short errorCode = partRespStruct.getShort("error_code");
-                long offset = partRespStruct.getLong("base_offset");
-                TopicPartition tp = new TopicPartition(topic, partition);
-                responses.put(tp, new PartitionResponse(errorCode, offset));
-            }
-        }
-    }
-
-    public Map<TopicPartition, PartitionResponse> responses() {
-        return this.responses;
-    }
-
-    public static final class PartitionResponse {
-        public short errorCode;
-        public long baseOffset;
-
-        public PartitionResponse(short errorCode, long baseOffset) {
-            this.errorCode = errorCode;
-            this.baseOffset = baseOffset;
-        }
-
-        @Override
-        public String toString() {
-            StringBuilder b = new StringBuilder();
-            b.append('{');
-            b.append("error: ");
-            b.append(errorCode);
-            b.append(",offset: ");
-            b.append(baseOffset);
-            b.append('}');
-            return b.toString();
-        }
-    }
-
-    public static ProduceResponse parse(ByteBuffer buffer) {
-        return new ProduceResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/RequestHeader.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/RequestHeader.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/RequestHeader.java
deleted file mode 100644
index ee345c5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/RequestHeader.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.protocol.ProtoUtils;
-import org.apache.kafka.copied.common.protocol.Protocol;
-import org.apache.kafka.copied.common.protocol.types.Field;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-/**
- * The header for a request in the Kafka protocol
- */
-public class RequestHeader extends AbstractRequestResponse {
-
-    private static final Field API_KEY_FIELD = Protocol.REQUEST_HEADER.get("api_key");
-    private static final Field API_VERSION_FIELD = Protocol.REQUEST_HEADER.get("api_version");
-    private static final Field CLIENT_ID_FIELD = Protocol.REQUEST_HEADER.get("client_id");
-    private static final Field CORRELATION_ID_FIELD = Protocol.REQUEST_HEADER.get("correlation_id");
-
-    private final short apiKey;
-    private final short apiVersion;
-    private final String clientId;
-    private final int correlationId;
-
-    public RequestHeader(Struct header) {
-        super(header);
-        apiKey = struct.getShort(API_KEY_FIELD);
-        apiVersion = struct.getShort(API_VERSION_FIELD);
-        clientId = struct.getString(CLIENT_ID_FIELD);
-        correlationId = struct.getInt(CORRELATION_ID_FIELD);
-    }
-
-    public RequestHeader(short apiKey, String client, int correlation) {
-        this(apiKey, ProtoUtils.latestVersion(apiKey), client, correlation);
-    }
-
-    public RequestHeader(short apiKey, short version, String client, int correlation) {
-        super(new Struct(Protocol.REQUEST_HEADER));
-        struct.set(API_KEY_FIELD, apiKey);
-        struct.set(API_VERSION_FIELD, version);
-        struct.set(CLIENT_ID_FIELD, client);
-        struct.set(CORRELATION_ID_FIELD, correlation);
-        this.apiKey = apiKey;
-        this.apiVersion = version;
-        this.clientId = client;
-        this.correlationId = correlation;
-    }
-
-    public short apiKey() {
-        return apiKey;
-    }
-
-    public short apiVersion() {
-        return apiVersion;
-    }
-
-    public String clientId() {
-        return clientId;
-    }
-
-    public int correlationId() {
-        return correlationId;
-    }
-
-    public static RequestHeader parse(ByteBuffer buffer) {
-        return new RequestHeader((Struct) Protocol.REQUEST_HEADER.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/RequestSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/RequestSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/RequestSend.java
deleted file mode 100644
index 0db1af6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/RequestSend.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.network.NetworkSend;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-/**
- * A send object for a kafka request
- */
-public class RequestSend extends NetworkSend {
-
-    private final RequestHeader header;
-    private final Struct body;
-
-    public RequestSend(String destination, RequestHeader header, Struct body) {
-        super(destination, serialize(header, body));
-        this.header = header;
-        this.body = body;
-    }
-
-    private static ByteBuffer serialize(RequestHeader header, Struct body) {
-        ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
-        header.writeTo(buffer);
-        body.writeTo(buffer);
-        buffer.rewind();
-        return buffer;
-    }
-
-    public RequestHeader header() {
-        return this.header;
-    }
-
-    public Struct body() {
-        return body;
-    }
-
-    @Override
-    public String toString() {
-        return "RequestSend(header=" + header.toString() + ", body=" + body.toString() + ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ResponseHeader.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ResponseHeader.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ResponseHeader.java
deleted file mode 100644
index 9ded109..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ResponseHeader.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.protocol.types.Field;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-import static org.apache.kafka.copied.common.protocol.Protocol.RESPONSE_HEADER;
-
-
-/**
- * A response header in the kafka protocol.
- */
-public class ResponseHeader extends AbstractRequestResponse {
-
-    private static final Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id");
-
-    private final int correlationId;
-
-    public ResponseHeader(Struct header) {
-        super(header);
-        correlationId = struct.getInt(CORRELATION_KEY_FIELD);
-    }
-
-    public ResponseHeader(int correlationId) {
-        super(new Struct(RESPONSE_HEADER));
-        struct.set(CORRELATION_KEY_FIELD, correlationId);
-        this.correlationId = correlationId;
-    }
-
-    public int correlationId() {
-        return correlationId;
-    }
-
-    public static ResponseHeader parse(ByteBuffer buffer) {
-        return new ResponseHeader((Struct) RESPONSE_HEADER.read(buffer));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ResponseSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ResponseSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ResponseSend.java
deleted file mode 100644
index 08363d6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/requests/ResponseSend.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.kafka.copied.common.requests;
-
-import org.apache.kafka.copied.common.network.NetworkSend;
-import org.apache.kafka.copied.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-public class ResponseSend extends NetworkSend {
-
-    public ResponseSend(String destination, ResponseHeader header, Struct body) {
-        super(destination, serialize(header, body));
-    }
-
-    public ResponseSend(String destination, ResponseHeader header, AbstractRequestResponse response) {
-        this(destination, header, response.toStruct());
-    }
-
-    private static ByteBuffer serialize(ResponseHeader header, Struct body) {
-        ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
-        header.writeTo(buffer);
-        body.writeTo(buffer);
-        buffer.rewind();
-        return buffer;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/ByteArrayDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/ByteArrayDeserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/ByteArrayDeserializer.java
deleted file mode 100644
index 276db18..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/ByteArrayDeserializer.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.kafka.copied.common.serialization;
-
-import java.util.Map;
-
-public class ByteArrayDeserializer implements Deserializer<byte[]> {
-
-    @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        // nothing to do
-    }
-
-    @Override
-    public byte[] deserialize(String topic, byte[] data) {
-        return data;
-    }
-
-    @Override
-    public void close() {
-        // nothing to do
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/ByteArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/ByteArraySerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/ByteArraySerializer.java
deleted file mode 100644
index 1a364fd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/ByteArraySerializer.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.kafka.copied.common.serialization;
-
-import java.util.Map;
-
-public class ByteArraySerializer implements Serializer<byte[]> {
-
-    @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        // nothing to do
-    }
-
-    @Override
-    public byte[] serialize(String topic, byte[] data) {
-        return data;
-    }
-
-    @Override
-    public void close() {
-        // nothing to do
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/Deserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/Deserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/Deserializer.java
deleted file mode 100644
index baf56dc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/Deserializer.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.kafka.copied.common.serialization;
-
-import java.io.Closeable;
-import java.util.Map;
-
-/**
- *
- * @param <T> Type to be deserialized into.
- *
- * A class that implements this interface is expected to have a constructor with no parameter.
- */
-public interface Deserializer<T> extends Closeable {
-
-    /**
-     * Configure this class.
-     * @param configs configs in key/value pairs
-     * @param isKey whether is for key or value
-     */
-    public void configure(Map<String, ?> configs, boolean isKey);
-    
-    /**
-     *
-     * @param topic topic associated with the data
-     * @param data serialized bytes
-     * @return deserialized typed data
-     */
-    public T deserialize(String topic, byte[] data);
-
-    @Override
-    public void close();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/IntegerDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/IntegerDeserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/IntegerDeserializer.java
deleted file mode 100644
index f46077e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/IntegerDeserializer.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.kafka.copied.common.serialization;
-
-import org.apache.kafka.copied.common.errors.SerializationException;
-
-import java.util.Map;
-
-public class IntegerDeserializer implements Deserializer<Integer> {
-
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        // nothing to do
-    }
-
-    public Integer deserialize(String topic, byte[] data) {
-        if (data == null)
-            return null;
-        if (data.length != 4) {
-            throw new SerializationException("Size of data received by IntegerDeserializer is " +
-                    "not 4");
-        }
-
-        int value = 0;
-        for (byte b : data) {
-            value <<= 8;
-            value |= b & 0xFF;
-        }
-        return value;
-    }
-
-    public void close() {
-        // nothing to do
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/IntegerSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/IntegerSerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/IntegerSerializer.java
deleted file mode 100644
index f333690..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/IntegerSerializer.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.kafka.copied.common.serialization;
-
-import java.util.Map;
-
-public class IntegerSerializer implements Serializer<Integer> {
-
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        // nothing to do
-    }
-
-    public byte[] serialize(String topic, Integer data) {
-        if (data == null)
-            return null;
-
-        return new byte[] {
-            (byte) (data >>> 24),
-            (byte) (data >>> 16),
-            (byte) (data >>> 8),
-            data.byteValue()
-        };
-    }
-
-    public void close() {
-        // nothing to do
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/Serializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/Serializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/Serializer.java
deleted file mode 100644
index 575c2e0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/Serializer.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.kafka.copied.common.serialization;
-
-import java.io.Closeable;
-import java.util.Map;
-
-/**
- *
- * @param <T> Type to be serialized from.
- *
- * A class that implements this interface is expected to have a constructor with no parameter.
- */
-public interface Serializer<T> extends Closeable {
-
-    /**
-     * Configure this class.
-     * @param configs configs in key/value pairs
-     * @param isKey whether is for key or value
-     */
-    public void configure(Map<String, ?> configs, boolean isKey);
-
-    /**
-     * @param topic topic associated with data
-     * @param data typed data
-     * @return serialized bytes
-     */
-    public byte[] serialize(String topic, T data);
-
-
-    /**
-     * Close this serializer.
-     * This method has to be idempotent if the serializer is used in KafkaProducer because it might be called
-     * multiple times.
-     */
-    @Override
-    public void close();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/StringDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/StringDeserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/StringDeserializer.java
deleted file mode 100644
index 2e17c9b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/serialization/StringDeserializer.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.kafka.copied.common.serialization;
-
-import org.apache.kafka.copied.common.errors.SerializationException;
-
-import java.io.UnsupportedEncodingException;
-import java.util.Map;
-
-/**
- *  String encoding defaults to UTF8 and can be customized by setting the property key.deserializer.encoding,
- *  value.deserializer.encoding or deserializer.encoding. The first two take precedence over the last.
- */
-public class StringDeserializer implements Deserializer<String> {
-    private String encoding = "UTF8";
-
-    @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        String propertyName = isKey ? "key.deserializer.encoding" : "value.deserializer.encoding";
-        Object encodingValue = configs.get(propertyName);
-        if (encodingValue == null)
-            encodingValue = configs.get("deserializer.encoding");
-        if (encodingValue != null && encodingValue instanceof String)
-            encoding = (String) encodingValue;
-    }
-
-    @Override
-    public String deserialize(String topic, byte[] data) {
-        try {
-            if (data == null)
-                return null;
-            else
-                return new String(data, encoding);
-        } catch (UnsupportedEncodingException e) {
-            throw new SerializationException("Error when deserializing byte[] to string due to unsupported encoding " + encoding);
-        }
-    }
-
-    @Override
-    public void close() {
-        // nothing to do
-    }
-}


[17/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkReceive.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkReceive.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkReceive.java
new file mode 100644
index 0000000..c0d5d99
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkReceive.java
@@ -0,0 +1,128 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.network;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.ScatteringByteChannel;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A size delimited Receive that consists of a 4 byte network-ordered size N followed by N bytes of content
+ */
+public class NetworkReceive implements Receive {
+
+    public final static String UNKNOWN_SOURCE = "";
+    public final static int UNLIMITED = -1;
+
+    private final String source;
+    private final ByteBuffer size;
+    private final int maxSize;
+    private ByteBuffer buffer;
+
+
+    public NetworkReceive(String source, ByteBuffer buffer) {
+        this.source = source;
+        this.buffer = buffer;
+        this.size = null;
+        this.maxSize = UNLIMITED;
+    }
+
+    public NetworkReceive(String source) {
+        this.source = source;
+        this.size = ByteBuffer.allocate(4);
+        this.buffer = null;
+        this.maxSize = UNLIMITED;
+    }
+
+    public NetworkReceive(int maxSize, String source) {
+        this.source = source;
+        this.size = ByteBuffer.allocate(4);
+        this.buffer = null;
+        this.maxSize = maxSize;
+    }
+
+    public NetworkReceive() {
+        this(UNKNOWN_SOURCE);
+    }
+
+    @Override
+    public String source() {
+        return source;
+    }
+
+    @Override
+    public boolean complete() {
+        return !size.hasRemaining() && !buffer.hasRemaining();
+    }
+
+    public long readFrom(ScatteringByteChannel channel) throws IOException {
+        return readFromReadableChannel(channel);
+    }
+
+    // Need a method to read from ReadableByteChannel because BlockingChannel requires read with timeout
+    // See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work
+    // This can go away after we get rid of BlockingChannel
+    @Deprecated
+    public long readFromReadableChannel(ReadableByteChannel channel) throws IOException {
+        int read = 0;
+        if (size.hasRemaining()) {
+            int bytesRead = channel.read(size);
+            if (bytesRead < 0)
+                throw new EOFException();
+            read += bytesRead;
+            if (!size.hasRemaining()) {
+                size.rewind();
+                int receiveSize = size.getInt();
+                if (receiveSize < 0)
+                    throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + ")");
+                if (maxSize != UNLIMITED && receiveSize > maxSize)
+                    throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + " larger than " + maxSize + ")");
+                this.buffer = ByteBuffer.allocate(receiveSize);
+            }
+        }
+        if (buffer != null) {
+            int bytesRead = channel.read(buffer);
+            if (bytesRead < 0)
+                throw new EOFException();
+            read += bytesRead;
+        }
+
+        return read;
+    }
+
+    public ByteBuffer payload() {
+        return this.buffer;
+    }
+
+    // Used only by BlockingChannel, so we may be able to get rid of this when/if we get rid of BlockingChannel
+    @Deprecated
+    public long readCompletely(ReadableByteChannel channel) throws IOException {
+        int totalRead = 0;
+        while (!complete()) {
+            totalRead += readFromReadableChannel(channel);
+        }
+        return totalRead;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkSend.java
new file mode 100644
index 0000000..29ce09d
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/NetworkSend.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.kafka_backport.common.network;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A size delimited Send that consists of a 4 byte network-ordered size N followed by N bytes of content
+ */
+public class NetworkSend extends ByteBufferSend {
+
+    public NetworkSend(String destination, ByteBuffer... buffers) {
+        super(destination, sizeDelimit(buffers));
+    }
+
+    private static ByteBuffer[] sizeDelimit(ByteBuffer[] buffers) {
+        int size = 0;
+        for (int i = 0; i < buffers.length; i++)
+            size += buffers[i].remaining();
+        ByteBuffer[] delimited = new ByteBuffer[buffers.length + 1];
+        delimited[0] = ByteBuffer.allocate(4);
+        delimited[0].putInt(size);
+        delimited[0].rewind();
+        System.arraycopy(buffers, 0, delimited, 1, buffers.length);
+        return delimited;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Receive.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Receive.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Receive.java
new file mode 100644
index 0000000..b799e7c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Receive.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.kafka_backport.common.network;
+
+import java.io.IOException;
+import java.nio.channels.ScatteringByteChannel;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This interface models the in-progress reading of data from a channel to a source identified by an integer id
+ */
+public interface Receive {
+
+    /**
+     * The numeric id of the source from which we are receiving data.
+     */
+    public String source();
+
+    /**
+     * Are we done receiving data?
+     */
+    public boolean complete();
+
+    /**
+     * Read bytes into this receive from the given channel
+     * @param channel The channel to read from
+     * @return The number of bytes read
+     * @throws IOException If the reading fails
+     */
+    public long readFrom(ScatteringByteChannel channel) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selectable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selectable.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selectable.java
new file mode 100644
index 0000000..08da141
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selectable.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.kafka_backport.common.network;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * An interface for asynchronous, multi-channel network I/O
+ */
+public interface Selectable {
+
+    /**
+     * Begin establishing a socket connection to the given address identified by the given address
+     * @param id The id for this connection
+     * @param address The address to connect to
+     * @param sendBufferSize The send buffer for the socket
+     * @param receiveBufferSize The receive buffer for the socket
+     * @throws IOException If we cannot begin connecting
+     */
+    public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException;
+
+    /**
+     * Begin disconnecting the connection identified by the given id
+     */
+    public void disconnect(String id);
+
+    /**
+     * Wakeup this selector if it is blocked on I/O
+     */
+    public void wakeup();
+
+    /**
+     * Close this selector
+     */
+    public void close();
+
+    /**
+     * Queue the given request for sending in the subsequent {@poll(long)} calls
+     * @param send The request to send
+     */
+    public void send(Send send);
+
+    /**
+     * Do I/O. Reads, writes, connection establishment, etc.
+     * @param timeout The amount of time to block if there is nothing to do
+     * @throws IOException
+     */
+    public void poll(long timeout) throws IOException;
+
+    /**
+     * The list of sends that completed on the last {@link #poll(long, List) poll()} call.
+     */
+    public List<Send> completedSends();
+
+    /**
+     * The list of receives that completed on the last {@link #poll(long, List) poll()} call.
+     */
+    public List<NetworkReceive> completedReceives();
+
+    /**
+     * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()}
+     * call.
+     */
+    public List<String> disconnected();
+
+    /**
+     * The list of connections that completed their connection on the last {@link #poll(long, List) poll()}
+     * call.
+     */
+    public List<String> connected();
+
+    /**
+     * Disable reads from the given connection
+     * @param id The id for the connection
+     */
+    public void mute(String id);
+
+    /**
+     * Re-enable reads from the given connection
+     * @param id The id for the connection
+     */
+    public void unmute(String id);
+
+    /**
+     * Disable reads from all connections
+     */
+    public void muteAll();
+
+    /**
+     * Re-enable reads from all connections
+     */
+    public void unmuteAll();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selector.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selector.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selector.java
new file mode 100644
index 0000000..a886e3b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Selector.java
@@ -0,0 +1,664 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.network;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+import org.apache.flink.kafka_backport.common.metrics.Measurable;
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.apache.flink.kafka_backport.common.metrics.Metrics;
+import org.apache.flink.kafka_backport.common.metrics.Sensor;
+import org.apache.flink.kafka_backport.common.metrics.stats.Avg;
+import org.apache.flink.kafka_backport.common.metrics.stats.Count;
+import org.apache.flink.kafka_backport.common.metrics.stats.Max;
+import org.apache.flink.kafka_backport.common.metrics.stats.Rate;
+import org.apache.flink.kafka_backport.common.utils.SystemTime;
+import org.apache.flink.kafka_backport.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.channels.CancelledKeyException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.nio.channels.UnresolvedAddressException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A nioSelector interface for doing non-blocking multi-connection network I/O.
+ * <p>
+ * This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and
+ * responses.
+ * <p>
+ * A connection can be added to the nioSelector associated with an integer id by doing
+ * 
+ * <pre>
+ * nioSelector.connect(42, new InetSocketAddress(&quot;google.com&quot;, server.port), 64000, 64000);
+ * </pre>
+ * 
+ * The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating
+ * the connection. The successful invocation of this method does not mean a valid connection has been established.
+ * 
+ * Sending requests, receiving responses, processing connection completions, and disconnections on the existing
+ * connections are all done using the <code>poll()</code> call.
+ * 
+ * <pre>
+ * List&lt;NetworkRequest&gt; requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
+ * nioSelector.poll(TIMEOUT_MS, requestsToSend);
+ * </pre>
+ * 
+ * The nioSelector maintains several lists that are reset by each call to <code>poll()</code> which are available via
+ * various getters. These are reset by each call to <code>poll()</code>.
+ * 
+ * This class is not thread safe!
+ */
+public class Selector implements Selectable {
+
+    private static final Logger log = LoggerFactory.getLogger(Selector.class);
+
+    private final java.nio.channels.Selector nioSelector;
+    private final Map<String, SelectionKey> keys;
+    private final List<Send> completedSends;
+    private final List<NetworkReceive> completedReceives;
+    private final List<String> disconnected;
+    private final List<String> connected;
+    private final List<String> failedSends;
+    private final Time time;
+    private final SelectorMetrics sensors;
+    private final String metricGrpPrefix;
+    private final Map<String, String> metricTags;
+    private final Map<String, Long> lruConnections;
+    private final long connectionsMaxIdleNanos;
+    private final int maxReceiveSize;
+    private final boolean metricsPerConnection;
+    private long currentTimeNanos;
+    private long nextIdleCloseCheckTime;
+
+
+    /**
+     * Create a new nioSelector
+     */
+    public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags, boolean metricsPerConnection) {
+        try {
+            this.nioSelector = java.nio.channels.Selector.open();
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+        this.maxReceiveSize = maxReceiveSize;
+        this.connectionsMaxIdleNanos = connectionMaxIdleMs * 1000 * 1000;
+        this.time = time;
+        this.metricGrpPrefix = metricGrpPrefix;
+        this.metricTags = metricTags;
+        this.keys = new HashMap<String, SelectionKey>();
+        this.completedSends = new ArrayList<Send>();
+        this.completedReceives = new ArrayList<NetworkReceive>();
+        this.connected = new ArrayList<String>();
+        this.disconnected = new ArrayList<String>();
+        this.failedSends = new ArrayList<String>();
+        this.sensors = new SelectorMetrics(metrics);
+        // initial capacity and load factor are default, we set them explicitly because we want to set accessOrder = true
+        this.lruConnections = new LinkedHashMap<String, Long>(16, .75F, true);
+        currentTimeNanos = new SystemTime().nanoseconds();
+        nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos;
+        this.metricsPerConnection = metricsPerConnection;
+    }
+
+    public Selector(long connectionMaxIdleMS, Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags) {
+        this(NetworkReceive.UNLIMITED, connectionMaxIdleMS, metrics, time, metricGrpPrefix, metricTags, true);
+    }
+
+    /**
+     * Begin connecting to the given address and add the connection to this nioSelector associated with the given id
+     * number.
+     * <p>
+     * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)}
+     * call. Check {@link #connected()} to see which (if any) connections have completed after a given poll call.
+     * @param id The id for the new connection
+     * @param address The address to connect to
+     * @param sendBufferSize The send buffer for the new connection
+     * @param receiveBufferSize The receive buffer for the new connection
+     * @throws IllegalStateException if there is already a connection for that id
+     * @throws IOException if DNS resolution fails on the hostname or if the broker is down
+     */
+    @Override
+    public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException {
+        if (this.keys.containsKey(id))
+            throw new IllegalStateException("There is already a connection for id " + id);
+
+        SocketChannel channel = SocketChannel.open();
+        channel.configureBlocking(false);
+        Socket socket = channel.socket();
+        socket.setKeepAlive(true);
+        socket.setSendBufferSize(sendBufferSize);
+        socket.setReceiveBufferSize(receiveBufferSize);
+        socket.setTcpNoDelay(true);
+        try {
+            channel.connect(address);
+        } catch (UnresolvedAddressException e) {
+            channel.close();
+            throw new IOException("Can't resolve address: " + address, e);
+        } catch (IOException e) {
+            channel.close();
+            throw e;
+        }
+        SelectionKey key = channel.register(this.nioSelector, SelectionKey.OP_CONNECT);
+        key.attach(new Transmissions(id));
+        this.keys.put(id, key);
+    }
+
+    /**
+     * Register the nioSelector with an existing channel
+     * Use this on server-side, when a connection is accepted by a different thread but processed by the Selector
+     * Note that we are not checking if the connection id is valid - since the connection already exists
+     */
+    public void register(String id, SocketChannel channel) throws ClosedChannelException {
+        SelectionKey key = channel.register(nioSelector, SelectionKey.OP_READ);
+        key.attach(new Transmissions(id));
+        this.keys.put(id, key);
+    }
+
+    /**
+     * Disconnect any connections for the given id (if there are any). The disconnection is asynchronous and will not be
+     * processed until the next {@link #poll(long, List) poll()} call.
+     */
+    @Override
+    public void disconnect(String id) {
+        SelectionKey key = this.keys.get(id);
+        if (key != null)
+            key.cancel();
+    }
+
+    /**
+     * Interrupt the nioSelector if it is blocked waiting to do I/O.
+     */
+    @Override
+    public void wakeup() {
+        this.nioSelector.wakeup();
+    }
+
+    /**
+     * Close this selector and all associated connections
+     */
+    @Override
+    public void close() {
+        List<String> connections = new LinkedList<String>(keys.keySet());
+        for (String id: connections)
+            close(id);
+
+        try {
+            this.nioSelector.close();
+        } catch (IOException e) {
+            log.error("Exception closing nioSelector:", e);
+        }
+    }
+
+    /**
+     * Queue the given request for sending in the subsequent {@poll(long)} calls
+     * @param send The request to send
+     */
+    public void send(Send send) {
+        SelectionKey key = keyForId(send.destination());
+        Transmissions transmissions = transmissions(key);
+        if (transmissions.hasSend())
+            throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress.");
+        transmissions.send = send;
+        try {
+            key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+        } catch (CancelledKeyException e) {
+            close(transmissions.id);
+            this.failedSends.add(send.destination());
+        }
+    }
+
+    /**
+     * Do whatever I/O can be done on each connection without blocking. This includes completing connections, completing
+     * disconnections, initiating new sends, or making progress on in-progress sends or receives.
+     * 
+     * When this call is completed the user can check for completed sends, receives, connections or disconnects using
+     * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These
+     * lists will be cleared at the beginning of each {@link #poll(long, List)} call and repopulated by the call if any
+     * completed I/O.
+     * 
+     * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely.
+     * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is
+     *         already an in-progress send
+     */
+    @Override
+    public void poll(long timeout) throws IOException {
+        clear();
+
+        /* check ready keys */
+        long startSelect = time.nanoseconds();
+        int readyKeys = select(timeout);
+        long endSelect = time.nanoseconds();
+        currentTimeNanos = endSelect;
+        this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds());
+
+        if (readyKeys > 0) {
+            Set<SelectionKey> keys = this.nioSelector.selectedKeys();
+            Iterator<SelectionKey> iter = keys.iterator();
+            while (iter.hasNext()) {
+                SelectionKey key = iter.next();
+                iter.remove();
+
+                Transmissions transmissions = transmissions(key);
+                SocketChannel channel = channel(key);
+
+                // register all per-connection metrics at once
+                sensors.maybeRegisterConnectionMetrics(transmissions.id);
+                lruConnections.put(transmissions.id, currentTimeNanos);
+
+                try {
+                    /* complete any connections that have finished their handshake */
+                    if (key.isConnectable()) {
+                        channel.finishConnect();
+                        key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ);
+                        this.connected.add(transmissions.id);
+                        this.sensors.connectionCreated.record();
+                        log.debug("Connection {} created", transmissions.id);
+                    }
+
+                    /* read from any connections that have readable data */
+                    if (key.isReadable()) {
+                        if (!transmissions.hasReceive())
+                            transmissions.receive = new NetworkReceive(maxReceiveSize, transmissions.id);
+                        try {
+                            transmissions.receive.readFrom(channel);
+                        } catch (InvalidReceiveException e) {
+                            log.error("Invalid data received from " + transmissions.id + " closing connection", e);
+                            close(transmissions.id);
+                            this.disconnected.add(transmissions.id);
+                            throw e;
+                        }
+                        if (transmissions.receive.complete()) {
+                            transmissions.receive.payload().rewind();
+                            this.completedReceives.add(transmissions.receive);
+                            this.sensors.recordBytesReceived(transmissions.id, transmissions.receive.payload().limit());
+                            transmissions.clearReceive();
+                        }
+                    }
+
+                    /* write to any sockets that have space in their buffer and for which we have data */
+                    if (key.isWritable()) {
+                        transmissions.send.writeTo(channel);
+                        if (transmissions.send.completed()) {
+                            this.completedSends.add(transmissions.send);
+                            this.sensors.recordBytesSent(transmissions.id, transmissions.send.size());
+                            transmissions.clearSend();
+                            key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
+                        }
+                    }
+
+                    /* cancel any defunct sockets */
+                    if (!key.isValid()) {
+                        close(transmissions.id);
+                        this.disconnected.add(transmissions.id);
+                    }
+                } catch (IOException e) {
+                    String desc = socketDescription(channel);
+                    if (e instanceof EOFException || e instanceof ConnectException)
+                        log.debug("Connection {} disconnected", desc);
+                    else
+                        log.warn("Error in I/O with connection to {}", desc, e);
+                    close(transmissions.id);
+                    this.disconnected.add(transmissions.id);
+                }
+            }
+        }
+        long endIo = time.nanoseconds();
+        this.sensors.ioTime.record(endIo - endSelect, time.milliseconds());
+        maybeCloseOldestConnection();
+    }
+
+    private String socketDescription(SocketChannel channel) {
+        Socket socket = channel.socket();
+        if (socket == null)
+            return "[unconnected socket]";
+        else if (socket.getInetAddress() != null)
+            return socket.getInetAddress().toString();
+        else
+            return socket.getLocalAddress().toString();
+    }
+
+    @Override
+    public List<Send> completedSends() {
+        return this.completedSends;
+    }
+
+    @Override
+    public List<NetworkReceive> completedReceives() {
+        return this.completedReceives;
+    }
+
+    @Override
+    public List<String> disconnected() {
+        return this.disconnected;
+    }
+
+    @Override
+    public List<String> connected() {
+        return this.connected;
+    }
+
+    @Override
+    public void mute(String id) {
+        mute(this.keyForId(id));
+    }
+
+    private void mute(SelectionKey key) {
+        key.interestOps(key.interestOps() & ~SelectionKey.OP_READ);
+    }
+
+    @Override
+    public void unmute(String id) {
+        unmute(this.keyForId(id));
+    }
+
+    private void unmute(SelectionKey key) {
+        key.interestOps(key.interestOps() | SelectionKey.OP_READ);
+    }
+
+    @Override
+    public void muteAll() {
+        for (SelectionKey key : this.keys.values())
+            mute(key);
+    }
+
+    @Override
+    public void unmuteAll() {
+        for (SelectionKey key : this.keys.values())
+            unmute(key);
+    }
+
+    private void maybeCloseOldestConnection() {
+        if (currentTimeNanos > nextIdleCloseCheckTime) {
+            if (lruConnections.isEmpty()) {
+                nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos;
+            } else {
+                Map.Entry<String, Long> oldestConnectionEntry = lruConnections.entrySet().iterator().next();
+                Long connectionLastActiveTime = oldestConnectionEntry.getValue();
+                nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos;
+                if (currentTimeNanos > nextIdleCloseCheckTime) {
+                    String connectionId = oldestConnectionEntry.getKey();
+                    if (log.isTraceEnabled())
+                        log.trace("About to close the idle connection from " + connectionId
+                                + " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis");
+
+                    disconnected.add(connectionId);
+                    close(connectionId);
+                }
+            }
+        }
+    }
+
+    /**
+     * Clear the results from the prior poll
+     */
+    private void clear() {
+        this.completedSends.clear();
+        this.completedReceives.clear();
+        this.connected.clear();
+        this.disconnected.clear();
+        this.disconnected.addAll(this.failedSends);
+        this.failedSends.clear();
+    }
+
+    /**
+     * Check for data, waiting up to the given timeout.
+     * 
+     * @param ms Length of time to wait, in milliseconds. If negative, wait indefinitely.
+     * @return The number of keys ready
+     * @throws IOException
+     */
+    private int select(long ms) throws IOException {
+        if (ms == 0L)
+            return this.nioSelector.selectNow();
+        else if (ms < 0L)
+            return this.nioSelector.select();
+        else
+            return this.nioSelector.select(ms);
+    }
+
+    /**
+     * Begin closing this connection
+     */
+    public void close(String id) {
+        SelectionKey key = keyForId(id);
+        lruConnections.remove(id);
+        SocketChannel channel = channel(key);
+        Transmissions trans = transmissions(key);
+        if (trans != null) {
+            this.keys.remove(trans.id);
+            trans.clearReceive();
+            trans.clearSend();
+        }
+        key.attach(null);
+        key.cancel();
+        try {
+            channel.socket().close();
+            channel.close();
+        } catch (IOException e) {
+            log.error("Exception closing connection to node {}:", trans.id, e);
+        }
+        this.sensors.connectionClosed.record();
+    }
+
+    /**
+     * Get the selection key associated with this numeric id
+     */
+    private SelectionKey keyForId(String id) {
+        SelectionKey key = this.keys.get(id);
+        if (key == null)
+            throw new IllegalStateException("Attempt to write to socket for which there is no open connection. Connection id " + id + " existing connections " + keys.keySet().toString());
+        return key;
+    }
+
+    /**
+     * Get the transmissions for the given connection
+     */
+    private Transmissions transmissions(SelectionKey key) {
+        return (Transmissions) key.attachment();
+    }
+
+    /**
+     * Get the socket channel associated with this selection key
+     */
+    private SocketChannel channel(SelectionKey key) {
+        return (SocketChannel) key.channel();
+    }
+
+    /**
+     * The id and in-progress send and receive associated with a connection
+     */
+    private static class Transmissions {
+        public String id;
+        public Send send;
+        public NetworkReceive receive;
+
+        public Transmissions(String id) {
+            this.id = id;
+        }
+
+        public boolean hasSend() {
+            return this.send != null;
+        }
+
+        public void clearSend() {
+            this.send = null;
+        }
+
+        public boolean hasReceive() {
+            return this.receive != null;
+        }
+
+        public void clearReceive() {
+            this.receive = null;
+        }
+    }
+
+    private class SelectorMetrics {
+        private final Metrics metrics;
+        public final Sensor connectionClosed;
+        public final Sensor connectionCreated;
+        public final Sensor bytesTransferred;
+        public final Sensor bytesSent;
+        public final Sensor bytesReceived;
+        public final Sensor selectTime;
+        public final Sensor ioTime;
+
+        public SelectorMetrics(Metrics metrics) {
+            this.metrics = metrics;
+            String metricGrpName = metricGrpPrefix + "-metrics";
+            StringBuilder tagsSuffix = new StringBuilder();
+
+            for (Map.Entry<String, String> tag: metricTags.entrySet()) {
+                tagsSuffix.append(tag.getKey());
+                tagsSuffix.append("-");
+                tagsSuffix.append(tag.getValue());
+            }
+
+            this.connectionClosed = this.metrics.sensor("connections-closed:" + tagsSuffix.toString());
+            MetricName metricName = new MetricName("connection-close-rate", metricGrpName, "Connections closed per second in the window.", metricTags);
+            this.connectionClosed.add(metricName, new Rate());
+
+            this.connectionCreated = this.metrics.sensor("connections-created:" + tagsSuffix.toString());
+            metricName = new MetricName("connection-creation-rate", metricGrpName, "New connections established per second in the window.", metricTags);
+            this.connectionCreated.add(metricName, new Rate());
+
+            this.bytesTransferred = this.metrics.sensor("bytes-sent-received:" + tagsSuffix.toString());
+            metricName = new MetricName("network-io-rate", metricGrpName, "The average number of network operations (reads or writes) on all connections per second.", metricTags);
+            bytesTransferred.add(metricName, new Rate(new Count()));
+
+            this.bytesSent = this.metrics.sensor("bytes-sent:" + tagsSuffix.toString(), bytesTransferred);
+            metricName = new MetricName("outgoing-byte-rate", metricGrpName, "The average number of outgoing bytes sent per second to all servers.", metricTags);
+            this.bytesSent.add(metricName, new Rate());
+            metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags);
+            this.bytesSent.add(metricName, new Rate(new Count()));
+            metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", metricTags);
+            this.bytesSent.add(metricName, new Avg());
+            metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", metricTags);
+            this.bytesSent.add(metricName, new Max());
+
+            this.bytesReceived = this.metrics.sensor("bytes-received:" + tagsSuffix.toString(), bytesTransferred);
+            metricName = new MetricName("incoming-byte-rate", metricGrpName, "Bytes/second read off all sockets", metricTags);
+            this.bytesReceived.add(metricName, new Rate());
+            metricName = new MetricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags);
+            this.bytesReceived.add(metricName, new Rate(new Count()));
+
+            this.selectTime = this.metrics.sensor("select-time:" + tagsSuffix.toString());
+            metricName = new MetricName("select-rate", metricGrpName, "Number of times the I/O layer checked for new I/O to perform per second", metricTags);
+            this.selectTime.add(metricName, new Rate(new Count()));
+            metricName = new MetricName("io-wait-time-ns-avg", metricGrpName, "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", metricTags);
+            this.selectTime.add(metricName, new Avg());
+            metricName = new MetricName("io-wait-ratio", metricGrpName, "The fraction of time the I/O thread spent waiting.", metricTags);
+            this.selectTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
+
+            this.ioTime = this.metrics.sensor("io-time:" + tagsSuffix.toString());
+            metricName = new MetricName("io-time-ns-avg", metricGrpName, "The average length of time for I/O per select call in nanoseconds.", metricTags);
+            this.ioTime.add(metricName, new Avg());
+            metricName = new MetricName("io-ratio", metricGrpName, "The fraction of time the I/O thread spent doing I/O", metricTags);
+            this.ioTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
+
+            metricName = new MetricName("connection-count", metricGrpName, "The current number of active connections.", metricTags);
+            this.metrics.addMetric(metricName, new Measurable() {
+                public double measure(MetricConfig config, long now) {
+                    return keys.size();
+                }
+            });
+        }
+
+        public void maybeRegisterConnectionMetrics(String connectionId) {
+            if (!connectionId.isEmpty() && metricsPerConnection) {
+                // if one sensor of the metrics has been registered for the connection,
+                // then all other sensors should have been registered; and vice versa
+                String nodeRequestName = "node-" + connectionId + ".bytes-sent";
+                Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
+                if (nodeRequest == null) {
+                    String metricGrpName = metricGrpPrefix + "-node-metrics";
+
+                    Map<String, String> tags = new LinkedHashMap<String, String>(metricTags);
+                    tags.put("node-id", "node-" + connectionId);
+
+                    nodeRequest = this.metrics.sensor(nodeRequestName);
+                    MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags);
+                    nodeRequest.add(metricName, new Rate());
+                    metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", tags);
+                    nodeRequest.add(metricName, new Rate(new Count()));
+                    metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", tags);
+                    nodeRequest.add(metricName, new Avg());
+                    metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", tags);
+                    nodeRequest.add(metricName, new Max());
+
+                    String nodeResponseName = "node-" + connectionId + ".bytes-received";
+                    Sensor nodeResponse = this.metrics.sensor(nodeResponseName);
+                    metricName = new MetricName("incoming-byte-rate", metricGrpName, tags);
+                    nodeResponse.add(metricName, new Rate());
+                    metricName = new MetricName("response-rate", metricGrpName, "The average number of responses received per second.", tags);
+                    nodeResponse.add(metricName, new Rate(new Count()));
+
+                    String nodeTimeName = "node-" + connectionId + ".latency";
+                    Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName);
+                    metricName = new MetricName("request-latency-avg", metricGrpName, tags);
+                    nodeRequestTime.add(metricName, new Avg());
+                    metricName = new MetricName("request-latency-max", metricGrpName, tags);
+                    nodeRequestTime.add(metricName, new Max());
+                }
+            }
+        }
+
+        public void recordBytesSent(String connectionId, long bytes) {
+            long now = time.milliseconds();
+            this.bytesSent.record(bytes, now);
+            if (!connectionId.isEmpty()) {
+                String nodeRequestName = "node-" + connectionId + ".bytes-sent";
+                Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
+                if (nodeRequest != null)
+                    nodeRequest.record(bytes, now);
+            }
+        }
+
+        public void recordBytesReceived(String connection, int bytes) {
+            long now = time.milliseconds();
+            this.bytesReceived.record(bytes, now);
+            if (!connection.isEmpty()) {
+                String nodeRequestName = "node-" + connection + ".bytes-received";
+                Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
+                if (nodeRequest != null)
+                    nodeRequest.record(bytes, now);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Send.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Send.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Send.java
new file mode 100644
index 0000000..b9e8a50
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/network/Send.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.kafka_backport.common.network;
+
+import java.io.IOException;
+import java.nio.channels.GatheringByteChannel;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This interface models the in-progress sending of data to a destination identified by an integer id.
+ */
+public interface Send {
+
+    /**
+     * The numeric id for the destination of this send
+     */
+    public String destination();
+
+    /**
+     * Is this send complete?
+     */
+    public boolean completed();
+
+    /**
+     * Write some as-yet unwritten bytes from this send to the provided channel. It may take multiple calls for the send
+     * to be completely written
+     * @param channel The channel to write to
+     * @return The number of bytes written
+     * @throws IOException If the write fails
+     */
+    public long writeTo(GatheringByteChannel channel) throws IOException;
+
+    /**
+     * Size of the send
+     */
+    public long size();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ApiKeys.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ApiKeys.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ApiKeys.java
new file mode 100644
index 0000000..e12261c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ApiKeys.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.kafka_backport.common.protocol;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Identifiers for all the Kafka APIs
+ */
+public enum ApiKeys {
+    PRODUCE(0, "Produce"),
+    FETCH(1, "Fetch"),
+    LIST_OFFSETS(2, "Offsets"),
+    METADATA(3, "Metadata"),
+    LEADER_AND_ISR(4, "LeaderAndIsr"),
+    STOP_REPLICA(5, "StopReplica"),
+    UPDATE_METADATA_KEY(6, "UpdateMetadata"),
+    CONTROLLED_SHUTDOWN_KEY(7, "ControlledShutdown"),
+    OFFSET_COMMIT(8, "OffsetCommit"),
+    OFFSET_FETCH(9, "OffsetFetch"),
+    CONSUMER_METADATA(10, "ConsumerMetadata"),
+    JOIN_GROUP(11, "JoinGroup"),
+    HEARTBEAT(12, "Heartbeat");
+
+    private static ApiKeys[] codeToType;
+    public static final int MAX_API_KEY;
+
+    static {
+        int maxKey = -1;
+        for (ApiKeys key : ApiKeys.values()) {
+            maxKey = Math.max(maxKey, key.id);
+        }
+        codeToType = new ApiKeys[maxKey + 1];
+        for (ApiKeys key : ApiKeys.values()) {
+            codeToType[key.id] = key;
+        }
+        MAX_API_KEY = maxKey;
+    }
+
+    /** the perminant and immutable id of an API--this can't change ever */
+    public final short id;
+
+    /** an english description of the api--this is for debugging and can change */
+    public final String name;
+
+    private ApiKeys(int id, String name) {
+        this.id = (short) id;
+        this.name = name;
+    }
+
+    public static ApiKeys forId(int id) {
+        return codeToType[id];
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Errors.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Errors.java
new file mode 100644
index 0000000..5ef3b24
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Errors.java
@@ -0,0 +1,172 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.protocol;
+
+import org.apache.flink.kafka_backport.common.errors.ApiException;
+import org.apache.flink.kafka_backport.common.errors.IllegalGenerationException;
+import org.apache.flink.kafka_backport.common.errors.InvalidRequiredAcksException;
+import org.apache.flink.kafka_backport.common.errors.InvalidTopicException;
+import org.apache.flink.kafka_backport.common.errors.LeaderNotAvailableException;
+import org.apache.flink.kafka_backport.common.errors.NetworkException;
+import org.apache.flink.kafka_backport.common.errors.NotCoordinatorForConsumerException;
+import org.apache.flink.kafka_backport.common.errors.NotEnoughReplicasException;
+import org.apache.flink.kafka_backport.common.errors.NotLeaderForPartitionException;
+import org.apache.flink.kafka_backport.common.errors.OffsetLoadInProgressException;
+import org.apache.flink.kafka_backport.common.errors.OffsetMetadataTooLarge;
+import org.apache.flink.kafka_backport.common.errors.OffsetOutOfRangeException;
+import org.apache.flink.kafka_backport.common.errors.RecordBatchTooLargeException;
+import org.apache.flink.kafka_backport.common.errors.TimeoutException;
+import org.apache.flink.kafka_backport.common.errors.UnknownConsumerIdException;
+import org.apache.flink.kafka_backport.common.errors.UnknownServerException;
+import org.apache.flink.kafka_backport.common.errors.UnknownTopicOrPartitionException;
+import org.apache.flink.kafka_backport.common.errors.ConsumerCoordinatorNotAvailableException;
+import org.apache.flink.kafka_backport.common.errors.CorruptRecordException;
+import org.apache.flink.kafka_backport.common.errors.NotEnoughReplicasAfterAppendException;
+import org.apache.flink.kafka_backport.common.errors.RecordTooLargeException;
+
+import java.util.HashMap;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This class contains all the client-server errors--those errors that must be sent from the server to the client. These
+ * are thus part of the protocol. The names can be changed but the error code cannot.
+ * 
+ * Do not add exceptions that occur only on the client or only on the server here.
+ */
+public enum Errors {
+    UNKNOWN(-1, new UnknownServerException("The server experienced an unexpected error when processing the request")),
+    NONE(0, null),
+    OFFSET_OUT_OF_RANGE(1,
+            new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")),
+    CORRUPT_MESSAGE(2,
+            new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")),
+    UNKNOWN_TOPIC_OR_PARTITION(3,
+            new UnknownTopicOrPartitionException("This server does not host this topic-partition.")),
+    // TODO: errorCode 4 for InvalidFetchSize
+    LEADER_NOT_AVAILABLE(5,
+            new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")),
+    NOT_LEADER_FOR_PARTITION(6,
+            new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")),
+    REQUEST_TIMED_OUT(7,
+            new TimeoutException("The request timed out.")),
+    // TODO: errorCode 8 for BrokerNotAvailable
+    REPLICA_NOT_AVAILABLE(9,
+            new ApiException("The replica is not available for the requested topic-partition")),
+    MESSAGE_TOO_LARGE(10,
+            new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")),
+    OFFSET_METADATA_TOO_LARGE(12,
+            new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")),
+    NETWORK_EXCEPTION(13,
+            new NetworkException("The server disconnected before a response was received.")),
+    OFFSET_LOAD_IN_PROGRESS(14,
+            new OffsetLoadInProgressException("The coordinator is loading offsets and can't process requests.")),
+    CONSUMER_COORDINATOR_NOT_AVAILABLE(15,
+            new ConsumerCoordinatorNotAvailableException("The coordinator is not available.")),
+    NOT_COORDINATOR_FOR_CONSUMER(16,
+            new NotCoordinatorForConsumerException("This is not the correct coordinator for this consumer.")),
+    INVALID_TOPIC_EXCEPTION(17,
+            new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")),
+    RECORD_LIST_TOO_LARGE(18,
+            new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")),
+    NOT_ENOUGH_REPLICAS(19,
+            new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")),
+    NOT_ENOUGH_REPLICAS_AFTER_APPEND(20,
+            new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")),
+    INVALID_REQUIRED_ACKS(21,
+            new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")),
+    ILLEGAL_GENERATION(22,
+            new IllegalGenerationException("Specified consumer generation id is not valid.")),
+    INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23,
+            new ApiException("The request partition assignment strategy does not match that of the group.")),
+    UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24,
+            new ApiException("The request partition assignment strategy is unknown to the broker.")),
+    UNKNOWN_CONSUMER_ID(25,
+            new UnknownConsumerIdException("The coordinator is not aware of this consumer.")),
+    INVALID_SESSION_TIMEOUT(26,
+            new ApiException("The session timeout is not within an acceptable range.")),
+    COMMITTING_PARTITIONS_NOT_ASSIGNED(27,
+            new ApiException("Some of the committing partitions are not assigned the committer")),
+    INVALID_COMMIT_OFFSET_SIZE(28,
+            new ApiException("The committing offset data size is not valid"));
+
+    private static Map<Class<?>, Errors> classToError = new HashMap<Class<?>, Errors>();
+    private static Map<Short, Errors> codeToError = new HashMap<Short, Errors>();
+
+    static {
+        for (Errors error : Errors.values()) {
+            codeToError.put(error.code(), error);
+            if (error.exception != null)
+                classToError.put(error.exception.getClass(), error);
+        }
+    }
+
+    private final short code;
+    private final ApiException exception;
+
+    private Errors(int code, ApiException exception) {
+        this.code = (short) code;
+        this.exception = exception;
+    }
+
+    /**
+     * An instance of the exception
+     */
+    public ApiException exception() {
+        return this.exception;
+    }
+
+    /**
+     * The error code for the exception
+     */
+    public short code() {
+        return this.code;
+    }
+
+    /**
+     * Throw the exception corresponding to this error if there is one
+     */
+    public void maybeThrow() {
+        if (exception != null) {
+            throw this.exception;
+        }
+    }
+
+    /**
+     * Throw the exception if there is one
+     */
+    public static Errors forCode(short code) {
+        Errors error = codeToError.get(code);
+        return error == null ? UNKNOWN : error;
+    }
+
+    /**
+     * Return the error instance associated with this exception (or UKNOWN if there is none)
+     */
+    public static Errors forException(Throwable t) {
+        Errors error = classToError.get(t.getClass());
+        return error == null ? UNKNOWN : error;
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ProtoUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ProtoUtils.java
new file mode 100644
index 0000000..e5ae9a4
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/ProtoUtils.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.kafka_backport.common.protocol;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ProtoUtils {
+
+    private static Schema schemaFor(Schema[][] schemas, int apiKey, int version) {
+        if (apiKey < 0 || apiKey > schemas.length)
+            throw new IllegalArgumentException("Invalid api key: " + apiKey);
+        Schema[] versions = schemas[apiKey];
+        if (version < 0 || version > versions.length)
+            throw new IllegalArgumentException("Invalid version for API key " + apiKey + ": " + version);
+        return versions[version];
+    }
+
+    public static short latestVersion(int apiKey) {
+        if (apiKey < 0 || apiKey >= Protocol.CURR_VERSION.length)
+            throw new IllegalArgumentException("Invalid api key: " + apiKey);
+        return Protocol.CURR_VERSION[apiKey];
+    }
+
+    public static Schema requestSchema(int apiKey, int version) {
+        return schemaFor(Protocol.REQUESTS, apiKey, version);
+    }
+
+    public static Schema currentRequestSchema(int apiKey) {
+        return requestSchema(apiKey, latestVersion(apiKey));
+    }
+
+    public static Schema responseSchema(int apiKey, int version) {
+        return schemaFor(Protocol.RESPONSES, apiKey, version);
+    }
+
+    public static Schema currentResponseSchema(int apiKey) {
+        return schemaFor(Protocol.RESPONSES, apiKey, latestVersion(apiKey));
+    }
+
+    public static Struct parseRequest(int apiKey, int version, ByteBuffer buffer) {
+        return (Struct) requestSchema(apiKey, version).read(buffer);
+    }
+
+    public static Struct parseResponse(int apiKey, ByteBuffer buffer) {
+        return (Struct) currentResponseSchema(apiKey).read(buffer);
+    }
+
+}


[22/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/KafkaConsumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/KafkaConsumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/KafkaConsumer.java
new file mode 100644
index 0000000..8800954
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/KafkaConsumer.java
@@ -0,0 +1,1130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer;
+
+import org.apache.flink.kafka_backport.clients.Metadata;
+import org.apache.flink.kafka_backport.clients.consumer.internals.DelayedTask;
+import org.apache.flink.kafka_backport.common.KafkaException;
+import org.apache.flink.kafka_backport.common.metrics.MetricConfig;
+import org.apache.flink.kafka_backport.clients.ClientUtils;
+import org.apache.flink.kafka_backport.clients.NetworkClient;
+import org.apache.flink.kafka_backport.clients.consumer.internals.ConsumerNetworkClient;
+import org.apache.flink.kafka_backport.clients.consumer.internals.Coordinator;
+import org.apache.flink.kafka_backport.clients.consumer.internals.Fetcher;
+import org.apache.flink.kafka_backport.clients.consumer.internals.SubscriptionState;
+import org.apache.flink.kafka_backport.common.Cluster;
+import org.apache.flink.kafka_backport.common.Metric;
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.apache.flink.kafka_backport.common.PartitionInfo;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.metrics.JmxReporter;
+import org.apache.flink.kafka_backport.common.metrics.Metrics;
+import org.apache.flink.kafka_backport.common.metrics.MetricsReporter;
+import org.apache.flink.kafka_backport.common.network.Selector;
+import org.apache.flink.kafka_backport.common.serialization.Deserializer;
+import org.apache.flink.kafka_backport.common.utils.SystemTime;
+import org.apache.flink.kafka_backport.common.utils.Time;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.ConcurrentModificationException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.kafka_backport.common.utils.Utils.min;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A Kafka client that consumes records from a Kafka cluster.
+ * <p>
+ * It will transparently handle the failure of servers in the Kafka cluster, and transparently adapt as partitions of
+ * data it subscribes to migrate within the cluster. This client also interacts with the server to allow groups of
+ * consumers to load balance consumption using consumer groups (as described below).
+ * <p>
+ * The consumer maintains TCP connections to the necessary brokers to fetch data for the topics it subscribes to.
+ * Failure to close the consumer after use will leak these connections.
+ * <p>
+ * The consumer is not thread-safe. See <a href="#multithreaded">Multi-threaded Processing</a> for more details.
+ *
+ * <h3>Offsets and Consumer Position</h3>
+ * Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of
+ * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer
+ * which has position 5 has consumed records with offsets 0 through 4 and will next receive record with offset 5. There
+ * are actually two notions of position relevant to the user of the consumer.
+ * <p>
+ * The {@link #position(TopicPartition) position} of the consumer gives the offset of the next record that will be given
+ * out. It will be one larger than the highest offset the consumer has seen in that partition. It automatically advances
+ * every time the consumer receives data calls {@link #poll(long)} and receives messages.
+ * <p>
+ * The {@link #commit(CommitType) committed position} is the last offset that has been saved securely. Should the
+ * process fail and restart, this is the offset that it will recover to. The consumer can either automatically commit
+ * offsets periodically, or it can choose to control this committed position manually by calling
+ * {@link #commit(CommitType) commit}.
+ * <p>
+ * This distinction gives the consumer control over when a record is considered consumed. It is discussed in further
+ * detail below.
+ * 
+ * <h3>Consumer Groups</h3>
+ * 
+ * Kafka uses the concept of <i>consumer groups</i> to allow a pool of processes to divide up the work of consuming and
+ * processing records. These processes can either be running on the same machine or, as is more likely, they can be
+ * distributed over many machines to provide additional scalability and fault tolerance for processing.
+ * <p>
+ * Each Kafka consumer must specify a consumer group that it belongs to. Kafka will deliver each message in the
+ * subscribed topics to one process in each consumer group. This is achieved by balancing the partitions in the topic
+ * over the consumer processes in each group. So if there is a topic with four partitions, and a consumer group with two
+ * processes, each process would consume from two partitions. This group membership is maintained dynamically: if a
+ * process fails the partitions assigned to it will be reassigned to other processes in the same group, and if a new
+ * process joins the group, partitions will be moved from existing consumers to this new process.
+ * <p>
+ * So if two processes subscribe to a topic both specifying different groups they will each get all the records in that
+ * topic; if they both specify the same group they will each get about half the records.
+ * <p>
+ * Conceptually you can think of a consumer group as being a single logical subscriber that happens to be made up of
+ * multiple processes. As a multi-subscriber system, Kafka naturally supports having any number of consumer groups for a
+ * given topic without duplicating data (additional consumers are actually quite cheap).
+ * <p>
+ * This is a slight generalization of the functionality that is common in messaging systems. To get semantics similar to
+ * a queue in a traditional messaging system all processes would be part of a single consumer group and hence record
+ * delivery would be balanced over the group like with a queue. Unlike a traditional messaging system, though, you can
+ * have multiple such groups. To get semantics similar to pub-sub in a traditional messaging system each process would
+ * have it's own consumer group, so each process would subscribe to all the records published to the topic.
+ * <p>
+ * In addition, when offsets are committed they are always committed for a given consumer group.
+ * <p>
+ * It is also possible for the consumer to manually specify the partitions it subscribes to, which disables this dynamic
+ * partition balancing.
+ * 
+ * <h3>Usage Examples</h3>
+ * The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to
+ * demonstrate how to use them.
+ * 
+ * <h4>Simple Processing</h4>
+ * This example demonstrates the simplest usage of Kafka's consumer api.
+ * 
+ * <pre>
+ *     Properties props = new Properties();
+ *     props.put(&quot;bootstrap.servers&quot;, &quot;localhost:9092&quot;);
+ *     props.put(&quot;group.id&quot;, &quot;test&quot;);
+ *     props.put(&quot;enable.auto.commit&quot;, &quot;true&quot;);
+ *     props.put(&quot;auto.commit.interval.ms&quot;, &quot;1000&quot;);
+ *     props.put(&quot;session.timeout.ms&quot;, &quot;30000&quot;);
+ *     props.put(&quot;key.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
+ *     props.put(&quot;value.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
+ *     KafkaConsumer&lt;String, String&gt; consumer = new KafkaConsumer&lt;String, String&gt;(props);
+ *     consumer.subscribe(&quot;foo&quot;, &quot;bar&quot;);
+ *     while (true) {
+ *         ConsumerRecords&lt;String, String&gt; records = consumer.poll(100);
+ *         for (ConsumerRecord&lt;String, String&gt; record : records)
+ *             System.out.printf(&quot;offset = %d, key = %s, value = %s&quot;, record.offset(), record.key(), record.value());
+ *     }
+ * </pre>
+ * 
+ * Setting <code>enable.auto.commit</code> means that offsets are committed automatically with a frequency controlled by
+ * the config <code>auto.commit.interval.ms</code>.
+ * <p>
+ * The connection to the cluster is bootstrapped by specifying a list of one or more brokers to contact using the
+ * configuration <code>bootstrap.servers</code>. This list is just used to discover the rest of the brokers in the
+ * cluster and need not be an exhaustive list of servers in the cluster (though you may want to specify more than one in
+ * case there are servers down when the client is connecting).
+ * <p>
+ * In this example the client is subscribing to the topics <i>foo</i> and <i>bar</i> as part of a group of consumers
+ * called <i>test</i> as described above.
+ * <p>
+ * The broker will automatically detect failed processes in the <i>test</i> group by using a heartbeat mechanism. The
+ * consumer will automatically ping the cluster periodically, which let's the cluster know that it is alive. As long as
+ * the consumer is able to do this it is considered alive and retains the right to consume from the partitions assigned
+ * to it. If it stops heartbeating for a period of time longer than <code>session.timeout.ms</code> then it will be
+ * considered dead and it's partitions will be assigned to another process.
+ * <p>
+ * The deserializer settings specify how to turn bytes into objects. For example, by specifying string deserializers, we
+ * are saying that our record's key and value will just be simple strings.
+ * 
+ * <h4>Controlling When Messages Are Considered Consumed</h4>
+ * 
+ * In this example we will consume a batch of records and batch them up in memory, when we have sufficient records
+ * batched we will insert them into a database. If we allowed offsets to auto commit as in the previous example messages
+ * would be considered consumed after they were given out by the consumer, and it would be possible that our process
+ * could fail after we have read messages into our in-memory buffer but before they had been inserted into the database.
+ * To avoid this we will manually commit the offsets only once the corresponding messages have been inserted into the
+ * database. This gives us exact control of when a message is considered consumed. This raises the opposite possibility:
+ * the process could fail in the interval after the insert into the database but before the commit (even though this
+ * would likely just be a few milliseconds, it is a possibility). In this case the process that took over consumption
+ * would consume from last committed offset and would repeat the insert of the last batch of data. Used in this way
+ * Kafka provides what is often called "at-least once delivery" guarantees, as each message will likely be delivered one
+ * time but in failure cases could be duplicated.
+ * 
+ * <pre>
+ *     Properties props = new Properties();
+ *     props.put(&quot;bootstrap.servers&quot;, &quot;localhost:9092&quot;);
+ *     props.put(&quot;group.id&quot;, &quot;test&quot;);
+ *     props.put(&quot;enable.auto.commit&quot;, &quot;false&quot;);
+ *     props.put(&quot;auto.commit.interval.ms&quot;, &quot;1000&quot;);
+ *     props.put(&quot;session.timeout.ms&quot;, &quot;30000&quot;);
+ *     props.put(&quot;key.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
+ *     props.put(&quot;value.deserializer&quot;, &quot;org.apache.StringDeserializer&quot;);
+ *     KafkaConsumer&lt;String, String&gt; consumer = new KafkaConsumer&lt;String, String&gt;(props);
+ *     consumer.subscribe(&quot;foo&quot;, &quot;bar&quot;);
+ *     int commitInterval = 200;
+ *     List&lt;ConsumerRecord&lt;String, String&gt;&gt; buffer = new ArrayList&lt;ConsumerRecord&lt;String, String&gt;&gt;();
+ *     while (true) {
+ *         ConsumerRecords&lt;String, String&gt; records = consumer.poll(100);
+ *         for (ConsumerRecord&lt;String, String&gt; record : records) {
+ *             buffer.add(record);
+ *             if (buffer.size() &gt;= commitInterval) {
+ *                 insertIntoDb(buffer);
+ *                 consumer.commit(CommitType.SYNC);
+ *                 buffer.clear();
+ *             }
+ *         }
+ *     }
+ * </pre>
+ * 
+ * <h4>Subscribing To Specific Partitions</h4>
+ * 
+ * In the previous examples we subscribed to the topics we were interested in and let Kafka give our particular process
+ * a fair share of the partitions for those topics. This provides a simple load balancing mechanism so multiple
+ * instances of our program can divided up the work of processing records.
+ * <p>
+ * In this mode the consumer will just get the partitions it subscribes to and if the consumer instance fails no attempt
+ * will be made to rebalance partitions to other instances.
+ * <p>
+ * There are several cases where this makes sense:
+ * <ul>
+ * <li>The first case is if the process is maintaining some kind of local state associated with that partition (like a
+ * local on-disk key-value store) and hence it should only get records for the partition it is maintaining on disk.
+ * <li>Another case is if the process itself is highly available and will be restarted if it fails (perhaps using a
+ * cluster management framework like YARN, Mesos, or AWS facilities, or as part of a stream processing framework). In
+ * this case there is no need for Kafka to detect the failure and reassign the partition, rather the consuming process
+ * will be restarted on another machine.
+ * </ul>
+ * <p>
+ * This mode is easy to specify, rather than subscribing to the topic, the consumer just subscribes to particular
+ * partitions:
+ * 
+ * <pre>
+ *     String topic = &quot;foo&quot;;
+ *     TopicPartition partition0 = new TopicPartition(topic, 0);
+ *     TopicPartition partition1 = new TopicPartition(topic, 1);
+ *     consumer.subscribe(partition0);
+ *     consumer.subscribe(partition1);
+ * </pre>
+ * 
+ * The group that the consumer specifies is still used for committing offsets, but now the set of partitions will only
+ * be changed if the consumer specifies new partitions, and no attempt at failure detection will be made.
+ * <p>
+ * It isn't possible to mix both subscription to specific partitions (with no load balancing) and to topics (with load
+ * balancing) using the same consumer instance.
+ * 
+ * <h4>Managing Your Own Offsets</h4>
+ * 
+ * The consumer application need not use Kafka's built-in offset storage, it can store offsets in a store of it's own
+ * choosing. The primary use case for this is allowing the application to store both the offset and the results of the
+ * consumption in the same system in a way that both the results and offsets are stored atomically. This is not always
+ * possible, but when it is it will make the consumption fully atomic and give "exactly once" semantics that are
+ * stronger than the default "at-least once" semantics you get with Kafka's offset commit functionality.
+ * <p>
+ * Here are a couple of examples of this type of usage:
+ * <ul>
+ * <li>If the results of the consumption are being stored in a relational database, storing the offset in the database
+ * as well can allow committing both the results and offset in a single transaction. Thus either the transaction will
+ * succeed and the offset will be updated based on what was consumed or the result will not be stored and the offset
+ * won't be updated.
+ * <li>If the results are being stored in a local store it may be possible to store the offset there as well. For
+ * example a search index could be built by subscribing to a particular partition and storing both the offset and the
+ * indexed data together. If this is done in a way that is atomic, it is often possible to have it be the case that even
+ * if a crash occurs that causes unsync'd data to be lost, whatever is left has the corresponding offset stored as well.
+ * This means that in this case the indexing process that comes back having lost recent updates just resumes indexing
+ * from what it has ensuring that no updates are lost.
+ * </ul>
+ * 
+ * Each record comes with it's own offset, so to manage your own offset you just need to do the following:
+ * <ol>
+ * <li>Configure <code>enable.auto.commit=false</code>
+ * <li>Use the offset provided with each {@link ConsumerRecord} to save your position.
+ * <li>On restart restore the position of the consumer using {@link #seek(TopicPartition, long)}.
+ * </ol>
+ * 
+ * This type of usage is simplest when the partition assignment is also done manually (this would be likely in the
+ * search index use case described above). If the partition assignment is done automatically special care will also be
+ * needed to handle the case where partition assignments change. This can be handled using a special callback specified
+ * using <code>rebalance.callback.class</code>, which specifies an implementation of the interface
+ * {@link ConsumerRebalanceCallback}. When partitions are taken from a consumer the consumer will want to commit its
+ * offset for those partitions by implementing
+ * {@link ConsumerRebalanceCallback#onPartitionsRevoked(Consumer, Collection)}. When partitions are assigned to a
+ * consumer, the consumer will want to look up the offset for those new partitions an correctly initialize the consumer
+ * to that position by implementing {@link ConsumerRebalanceCallback#onPartitionsAssigned(Consumer, Collection)}.
+ * <p>
+ * Another common use for {@link ConsumerRebalanceCallback} is to flush any caches the application maintains for
+ * partitions that are moved elsewhere.
+ * 
+ * <h4>Controlling The Consumer's Position</h4>
+ * 
+ * In most use cases the consumer will simply consume records from beginning to end, periodically committing it's
+ * position (either automatically or manually). However Kafka allows the consumer to manually control it's position,
+ * moving forward or backwards in a partition at will. This means a consumer can re-consume older records, or skip to
+ * the most recent records without actually consuming the intermediate records.
+ * <p>
+ * There are several instances where manually controlling the consumer's position can be useful.
+ * <p>
+ * One case is for time-sensitive record processing it may make sense for a consumer that falls far enough behind to not
+ * attempt to catch up processing all records, but rather just skip to the most recent records.
+ * <p>
+ * Another use case is for a system that maintains local state as described in the previous section. In such a system
+ * the consumer will want to initialize it's position on start-up to whatever is contained in the local store. Likewise
+ * if the local state is destroyed (say because the disk is lost) the state may be recreated on a new machine by
+ * reconsuming all the data and recreating the state (assuming that Kafka is retaining sufficient history).
+ * 
+ * Kafka allows specifying the position using {@link #seek(TopicPartition, long)} to specify the new position. Special
+ * methods for seeking to the earliest and latest offset the server maintains are also available (
+ * {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively).
+ * 
+ *
+ * <h3><a name="multithreaded">Multi-threaded Processing</a></h3>
+ * 
+ * The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application
+ * making the call. It is the responsibility of the user to ensure that multi-threaded access
+ * is properly synchronized. Un-synchronized access will result in {@link ConcurrentModificationException}.
+ *
+ * <p>
+ * The only exception to this rule is {@link #wakeup()}, which can safely be used from an external thread to
+ * interrupt an active operation. In this case, a {@link ConsumerWakeupException} will be thrown from the thread
+ * blocking on the operation. This can be used to shutdown the consumer from another thread. The following
+ * snippet shows the typical pattern:
+ *
+ * <pre>
+ * public class KafkaConsumerRunner implements Runnable {
+ *     private final AtomicBoolean closed = new AtomicBoolean(false);
+ *     private final KafkaConsumer consumer;
+ *
+ *     public void run() {
+ *         try {
+ *             consumer.subscribe("topic");
+ *             while (!closed.get()) {
+ *                 ConsumerRecords records = consumer.poll(10000);
+ *                 // Handle new records
+ *             }
+ *         } catch (ConsumerWakeupException e) {
+ *             // Ignore exception if closing
+ *             if (!closed.get()) throw e;
+ *         } finally {
+ *             consumer.close();
+ *         }
+ *     }
+ *
+ *     // Shutdown hook which can be called from a separate thread
+ *     public void shutdown() {
+ *         closed.set(true);
+ *         consumer.wakeup();
+ *     }
+ * }
+ * </pre>
+ *
+ * Then in a separate thread, the consumer can be shutdown by setting the closed flag and waking up the consumer.
+ *
+ * <pre>
+ *     closed.set(true);
+ *     consumer.wakeup();
+ * </pre>
+ *
+ * <p>
+ * We have intentionally avoided implementing a particular threading model for processing. This leaves several
+ * options for implementing multi-threaded processing of records.
+ *
+ * 
+ * <h4>1. One Consumer Per Thread</h4>
+ * 
+ * A simple option is to give each thread it's own consumer instance. Here are the pros and cons of this approach:
+ * <ul>
+ * <li><b>PRO</b>: It is the easiest to implement
+ * <li><b>PRO</b>: It is often the fastest as no inter-thread co-ordination is needed
+ * <li><b>PRO</b>: It makes in-order processing on a per-partition basis very easy to implement (each thread just
+ * processes messages in the order it receives them).
+ * <li><b>CON</b>: More consumers means more TCP connections to the cluster (one per thread). In general Kafka handles
+ * connections very efficiently so this is generally a small cost.
+ * <li><b>CON</b>: Multiple consumers means more requests being sent to the server and slightly less batching of data
+ * which can cause some drop in I/O throughput.
+ * <li><b>CON</b>: The number of total threads across all processes will be limited by the total number of partitions.
+ * </ul>
+ * 
+ * <h4>2. Decouple Consumption and Processing</h4>
+ * 
+ * Another alternative is to have one or more consumer threads that do all data consumption and hands off
+ * {@link ConsumerRecords} instances to a blocking queue consumed by a pool of processor threads that actually handle
+ * the record processing.
+ * 
+ * This option likewise has pros and cons:
+ * <ul>
+ * <li><b>PRO</b>: This option allows independently scaling the number of consumers and processors. This makes it
+ * possible to have a single consumer that feeds many processor threads, avoiding any limitation on partitions.
+ * <li><b>CON</b>: Guaranteeing order across the processors requires particular care as the threads will execute
+ * independently an earlier chunk of data may actually be processed after a later chunk of data just due to the luck of
+ * thread execution timing. For processing that has no ordering requirements this is not a problem.
+ * <li><b>CON</b>: Manually committing the position becomes harder as it requires that all threads co-ordinate to ensure
+ * that processing is complete for that partition.
+ * </ul>
+ * 
+ * There are many possible variations on this approach. For example each processor thread can have it's own queue, and
+ * the consumer threads can hash into these queues using the TopicPartition to ensure in-order consumption and simplify
+ * commit.
+ * 
+ */
+public class KafkaConsumer<K, V> implements Consumer<K, V> {
+
+    private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class);
+    private static final long NO_CURRENT_THREAD = -1L;
+    private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1);
+
+    private final Coordinator coordinator;
+    private final Deserializer<K> keyDeserializer;
+    private final Deserializer<V> valueDeserializer;
+    private final Fetcher<K, V> fetcher;
+
+    private final Time time;
+    private final ConsumerNetworkClient client;
+    private final Metrics metrics;
+    private final SubscriptionState subscriptions;
+    private final Metadata metadata;
+    private final long retryBackoffMs;
+    private final boolean autoCommit;
+    private final long autoCommitIntervalMs;
+    private boolean closed = false;
+
+    // currentThread holds the threadId of the current thread accessing KafkaConsumer
+    // and is used to prevent multi-threaded access
+    private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD);
+    // refcount is used to allow reentrant access by the thread who has acquired currentThread
+    private final AtomicInteger refcount = new AtomicInteger(0);
+
+    // TODO: This timeout controls how long we should wait before retrying a request. We should be able
+    //       to leverage the work of KAFKA-2120 to get this value from configuration.
+    private long requestTimeoutMs = 5000L;
+
+    /**
+     * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
+     * are documented <a href="http://kafka.apache.org/documentation.html#consumerconfigs" >here</a>. Values can be
+     * either strings or objects of the appropriate type (for example a numeric configuration would accept either the
+     * string "42" or the integer 42).
+     * <p>
+     * Valid configuration strings are documented at {@link ConsumerConfig}
+     * 
+     * @param configs The consumer configs
+     */
+    public KafkaConsumer(Map<String, Object> configs) {
+        this(configs, null, null, null);
+    }
+
+    /**
+     * A consumer is instantiated by providing a set of key-value pairs as configuration, a
+     * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}.
+     * <p>
+     * Valid configuration strings are documented at {@link ConsumerConfig}
+     * 
+     * @param configs The consumer configs
+     * @param callback A callback interface that the user can implement to manage customized offsets on the start and
+     *            end of every rebalance operation.
+     * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method
+     *            won't be called in the consumer when the deserializer is passed in directly.
+     * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method
+     *            won't be called in the consumer when the deserializer is passed in directly.
+     */
+    public KafkaConsumer(Map<String, Object> configs,
+                         ConsumerRebalanceCallback callback,
+                         Deserializer<K> keyDeserializer,
+                         Deserializer<V> valueDeserializer) {
+        this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)),
+            callback,
+            keyDeserializer,
+            valueDeserializer);
+    }
+
+    /**
+     * A consumer is instantiated by providing a {@link Properties} object as configuration. Valid
+     * configuration strings are documented at {@link ConsumerConfig} A consumer is instantiated by providing a
+     * {@link Properties} object as configuration. Valid configuration strings are documented at
+     * {@link ConsumerConfig}
+     */
+    public KafkaConsumer(Properties properties) {
+        this(properties, null, null, null);
+    }
+
+    /**
+     * A consumer is instantiated by providing a {@link Properties} object as configuration and a
+     * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}.
+     * <p>
+     * Valid configuration strings are documented at {@link ConsumerConfig}
+     * 
+     * @param properties The consumer configuration properties
+     * @param callback A callback interface that the user can implement to manage customized offsets on the start and
+     *            end of every rebalance operation.
+     * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method
+     *            won't be called in the consumer when the deserializer is passed in directly.
+     * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method
+     *            won't be called in the consumer when the deserializer is passed in directly.
+     */
+    public KafkaConsumer(Properties properties,
+                         ConsumerRebalanceCallback callback,
+                         Deserializer<K> keyDeserializer,
+                         Deserializer<V> valueDeserializer) {
+        this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)),
+             callback,
+             keyDeserializer,
+             valueDeserializer);
+    }
+
+    @SuppressWarnings("unchecked")
+    private KafkaConsumer(ConsumerConfig config,
+                          ConsumerRebalanceCallback callback,
+                          Deserializer<K> keyDeserializer,
+                          Deserializer<V> valueDeserializer) {
+        try {
+            log.debug("Starting the Kafka consumer");
+            if (callback == null)
+                callback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
+                        ConsumerRebalanceCallback.class);
+            this.time = new SystemTime();
+            this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
+            this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
+
+            MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
+                    .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
+                            TimeUnit.MILLISECONDS);
+            String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
+            String jmxPrefix = "kafka.consumer";
+            if (clientId.length() <= 0)
+                clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement();
+            List<MetricsReporter> reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG,
+                    MetricsReporter.class);
+            reporters.add(new JmxReporter(jmxPrefix));
+            this.metrics = new Metrics(metricConfig, reporters, time);
+            this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
+            this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG));
+            List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
+            this.metadata.update(Cluster.bootstrap(addresses), 0);
+
+            String metricGrpPrefix = "consumer";
+            Map<String, String> metricsTags = new LinkedHashMap<String, String>();
+            metricsTags.put("client-id", clientId);
+            NetworkClient netClient = new NetworkClient(
+                    new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags),
+                    this.metadata,
+                    clientId,
+                    100, // a fixed large enough value will suffice
+                    config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG),
+                    config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG),
+                    config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG));
+            this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs);
+            OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase());
+            this.subscriptions = new SubscriptionState(offsetResetStrategy);
+            this.coordinator = new Coordinator(this.client,
+                    config.getString(ConsumerConfig.GROUP_ID_CONFIG),
+                    config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG),
+                    config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
+                    this.subscriptions,
+                    metrics,
+                    metricGrpPrefix,
+                    metricsTags,
+                    this.time,
+                    requestTimeoutMs,
+                    retryBackoffMs,
+                    wrapRebalanceCallback(callback));
+            if (keyDeserializer == null) {
+                this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
+                        Deserializer.class);
+                this.keyDeserializer.configure(config.originals(), true);
+            } else {
+                this.keyDeserializer = keyDeserializer;
+            }
+            if (valueDeserializer == null) {
+                this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
+                        Deserializer.class);
+                this.valueDeserializer.configure(config.originals(), false);
+            } else {
+                this.valueDeserializer = valueDeserializer;
+            }
+            this.fetcher = new Fetcher<K, V>(this.client,
+                    config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG),
+                    config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG),
+                    config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG),
+                    config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG),
+                    this.keyDeserializer,
+                    this.valueDeserializer,
+                    this.metadata,
+                    this.subscriptions,
+                    metrics,
+                    metricGrpPrefix,
+                    metricsTags,
+                    this.time,
+                    this.retryBackoffMs);
+
+            config.logUnused();
+
+            if (autoCommit)
+                scheduleAutoCommitTask(autoCommitIntervalMs);
+
+            log.debug("Kafka consumer created");
+        } catch (Throwable t) {
+            // call close methods if internal objects are already constructed
+            // this is to prevent resource leak. see KAFKA-2121
+            close(true);
+            // now propagate the exception
+            throw new KafkaException("Failed to construct kafka consumer", t);
+        }
+    }
+
+    /**
+     * The set of partitions currently assigned to this consumer. If subscription happened by directly subscribing to
+     * partitions using {@link #subscribe(TopicPartition...)} then this will simply return the list of partitions that
+     * were subscribed to. If subscription was done by specifying only the topic using {@link #subscribe(String...)}
+     * then this will give the set of topics currently assigned to the consumer (which may be none if the assignment
+     * hasn't happened yet, or the partitions are in the process of getting reassigned).
+     */
+    public Set<TopicPartition> subscriptions() {
+        acquire();
+        try {
+            return Collections.unmodifiableSet(this.subscriptions.assignedPartitions());
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Incrementally subscribes to the given list of topics and uses the consumer's group management functionality
+     * <p>
+     * As part of group management, the consumer will keep track of the list of consumers that belong to a particular
+     * group and will trigger a rebalance operation if one of the following events trigger -
+     * <ul>
+     * <li>Number of partitions change for any of the subscribed list of topics
+     * <li>Topic is created or deleted
+     * <li>An existing member of the consumer group dies
+     * <li>A new member is added to an existing consumer group via the join API
+     * </ul>
+     * 
+     * @param topics A variable list of topics that the consumer wants to subscribe to
+     */
+    @Override
+    public void subscribe(String... topics) {
+        acquire();
+        try {
+            log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", "));
+            for (String topic : topics)
+                this.subscriptions.subscribe(topic);
+            metadata.addTopics(topics);
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Incrementally subscribes to a specific topic partition and does not use the consumer's group management
+     * functionality. As such, there will be no rebalance operation triggered when group membership or cluster and topic
+     * metadata change.
+     * <p>
+     *
+     * @param partitions Partitions to incrementally subscribe to
+     */
+    @Override
+    public void subscribe(TopicPartition... partitions) {
+        acquire();
+        try {
+            log.debug("Subscribed to partitions(s): {}", Utils.join(partitions, ", "));
+            for (TopicPartition tp : partitions) {
+                this.subscriptions.subscribe(tp);
+                metadata.addTopics(tp.topic());
+            }
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Unsubscribe from the specific topics. This will trigger a rebalance operation and records for this topic will not
+     * be returned from the next {@link #poll(long) poll()} onwards
+     * 
+     * @param topics Topics to unsubscribe from
+     */
+    public void unsubscribe(String... topics) {
+        acquire();
+        try {
+            log.debug("Unsubscribed from topic(s): {}", Utils.join(topics, ", "));
+            // throw an exception if the topic was never subscribed to
+            for (String topic : topics)
+                this.subscriptions.unsubscribe(topic);
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Unsubscribe from the specific topic partitions. records for these partitions will not be returned from the next
+     * {@link #poll(long) poll()} onwards
+     * 
+     * @param partitions Partitions to unsubscribe from
+     */
+    public void unsubscribe(TopicPartition... partitions) {
+        acquire();
+        try {
+            log.debug("Unsubscribed from partitions(s): {}", Utils.join(partitions, ", "));
+            // throw an exception if the partition was never subscribed to
+            for (TopicPartition partition : partitions)
+                this.subscriptions.unsubscribe(partition);
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Fetches data for the topics or partitions specified using one of the subscribe APIs. It is an error to not have
+     * subscribed to any topics or partitions before polling for data.
+     * <p>
+     * The offset used for fetching the data is governed by whether or not {@link #seek(TopicPartition, long)} is used.
+     * If {@link #seek(TopicPartition, long)} is used, it will use the specified offsets on startup and on every
+     * rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed
+     * offset using {@link #commit(Map, CommitType) commit(offsets, sync)} for the subscribed list of partitions.
+     * 
+     * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, returns
+     *            immediately with any records available now. Must not be negative.
+     * @return map of topic to records since the last fetch for the subscribed list of topics and partitions
+     * 
+     * @throws NoOffsetForPartitionException If there is no stored offset for a subscribed partition and no automatic
+     *             offset reset policy has been configured.
+     */
+    @Override
+    public ConsumerRecords<K, V> poll(long timeout) {
+        acquire();
+        try {
+            if (timeout < 0)
+                throw new IllegalArgumentException("Timeout must not be negative");
+
+            // poll for new data until the timeout expires
+            long remaining = timeout;
+            while (remaining >= 0) {
+                long start = time.milliseconds();
+                Map<TopicPartition, List<ConsumerRecord<K, V>>> records = pollOnce(remaining);
+                long end = time.milliseconds();
+
+                if (!records.isEmpty()) {
+                    // if data is available, then return it, but first send off the
+                    // next round of fetches to enable pipelining while the user is
+                    // handling the fetched records.
+                    fetcher.initFetches(metadata.fetch());
+                    client.poll(0);
+                    return new ConsumerRecords<K, V>(records);
+                }
+
+                remaining -= end - start;
+
+                // nothing was available, so we should backoff before retrying
+                if (remaining > 0) {
+                    Utils.sleep(min(remaining, retryBackoffMs));
+                    remaining -= time.milliseconds() - end;
+                }
+            }
+
+            return ConsumerRecords.empty();
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Do one round of polling. In addition to checking for new data, this does any needed
+     * heart-beating, auto-commits, and offset updates.
+     * @param timeout The maximum time to block in the underlying poll
+     * @return The fetched records (may be empty)
+     */
+    private Map<TopicPartition, List<ConsumerRecord<K, V>>> pollOnce(long timeout) {
+        // TODO: Sub-requests should take into account the poll timeout (KAFKA-1894)
+        coordinator.ensureCoordinatorKnown();
+
+        // ensure we have partitions assigned if we expect to
+        if (subscriptions.partitionsAutoAssigned())
+            coordinator.ensurePartitionAssignment();
+
+        // fetch positions if we have partitions we're subscribed to that we
+        // don't know the offset for
+        if (!subscriptions.hasAllFetchPositions())
+            updateFetchPositions(this.subscriptions.missingFetchPositions());
+
+        // init any new fetches (won't resend pending fetches)
+        Cluster cluster = this.metadata.fetch();
+        fetcher.initFetches(cluster);
+        client.poll(timeout);
+        return fetcher.fetchedRecords();
+    }
+
+    private void scheduleAutoCommitTask(final long interval) {
+        DelayedTask task = new DelayedTask() {
+            public void run(long now) {
+                commit(CommitType.ASYNC);
+                client.schedule(this, now + interval);
+            }
+        };
+        client.schedule(task, time.milliseconds() + interval);
+    }
+
+    /**
+     * Commits the specified offsets for the specified list of topics and partitions to Kafka.
+     * <p>
+     * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every
+     * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
+     * should not be used.
+     * <p>
+     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
+     * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use
+     * {@link #commit(Map, CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC})
+     * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown
+     * to the caller).
+     *
+     * @param offsets The list of offsets per partition that should be committed to Kafka.
+     * @param commitType Control whether the commit is blocking
+     */
+    @Override
+    public void commit(final Map<TopicPartition, Long> offsets, CommitType commitType) {
+        commit(offsets, commitType, null);
+    }
+
+    /**
+     * Commits the specified offsets for the specified list of topics and partitions to Kafka.
+     * <p>
+     * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every
+     * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
+     * should not be used.
+     * <p>
+     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
+     * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e.
+     * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In
+     * this case, the error is either passed to the callback (if provided) or thrown to the caller.
+     *
+     * @param offsets The list of offsets per partition that should be committed to Kafka.
+     * @param commitType Control whether the commit is blocking
+     * @param callback Callback to invoke when the commit completes
+     */
+    @Override
+    public void commit(final Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
+        acquire();
+        try {
+            log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets);
+            coordinator.commitOffsets(offsets, commitType, callback);
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions.
+     * <p>
+     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
+     * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
+     * should not be used.
+     * <p>
+     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
+     * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e.
+     * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In
+     * this case, the error is either passed to the callback (if provided) or thrown to the caller.
+     *
+     * @param commitType Whether or not the commit should block until it is acknowledged.
+     * @param callback Callback to invoke when the commit completes
+     */
+    @Override
+    public void commit(CommitType commitType, ConsumerCommitCallback callback) {
+        acquire();
+        try {
+            // need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance)
+            Map<TopicPartition, Long> allConsumed = new HashMap<TopicPartition, Long>(this.subscriptions.allConsumed());
+            commit(allConsumed, commitType, callback);
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions.
+     * <p>
+     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
+     * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
+     * should not be used.
+     * <p>
+     * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
+     * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use
+     * {@link #commit(CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC})
+     * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown
+     * to the caller).
+     *
+     * @param commitType Whether or not the commit should block until it is acknowledged.
+     */
+    @Override
+    public void commit(CommitType commitType) {
+        commit(commitType, null);
+    }
+
+    /**
+     * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API
+     * is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that
+     * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets
+     */
+    @Override
+    public void seek(TopicPartition partition, long offset) {
+        acquire();
+        try {
+            log.debug("Seeking to offset {} for partition {}", offset, partition);
+            this.subscriptions.seek(partition, offset);
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Seek to the first offset for each of the given partitions
+     */
+    public void seekToBeginning(TopicPartition... partitions) {
+        acquire();
+        try {
+            Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
+                    : Arrays.asList(partitions);
+            for (TopicPartition tp : parts) {
+                log.debug("Seeking to beginning of partition {}", tp);
+                subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST);
+            }
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Seek to the last offset for each of the given partitions
+     */
+    public void seekToEnd(TopicPartition... partitions) {
+        acquire();
+        try {
+            Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
+                    : Arrays.asList(partitions);
+            for (TopicPartition tp : parts) {
+                log.debug("Seeking to end of partition {}", tp);
+                subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST);
+            }
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Returns the offset of the <i>next record</i> that will be fetched (if a record with that offset exists).
+     * 
+     * @param partition The partition to get the position for
+     * @return The offset
+     * @throws NoOffsetForPartitionException If a position hasn't been set for a given partition, and no reset policy is
+     *             available.
+     */
+    public long position(TopicPartition partition) {
+        acquire();
+        try {
+            if (!this.subscriptions.assignedPartitions().contains(partition))
+                throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer.");
+            Long offset = this.subscriptions.consumed(partition);
+            if (offset == null) {
+                updateFetchPositions(Collections.singleton(partition));
+                return this.subscriptions.consumed(partition);
+            } else {
+                return offset;
+            }
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Fetches the last committed offset for the given partition (whether the commit happened by this process or
+     * another). This offset will be used as the position for the consumer in the event of a failure.
+     * <p>
+     * This call may block to do a remote call if the partition in question isn't assigned to this consumer or if the
+     * consumer hasn't yet initialized it's cache of committed offsets.
+     * 
+     * @param partition The partition to check
+     * @return The last committed offset or null if no offset has been committed
+     * @throws NoOffsetForPartitionException If no offset has ever been committed by any process for the given
+     *             partition.
+     */
+    @Override
+    public long committed(TopicPartition partition) {
+        acquire();
+        try {
+            Long committed;
+            if (subscriptions.assignedPartitions().contains(partition)) {
+                committed = this.subscriptions.committed(partition);
+                if (committed == null) {
+                    coordinator.refreshCommittedOffsetsIfNeeded();
+                    committed = this.subscriptions.committed(partition);
+                }
+            } else {
+                Map<TopicPartition, Long> offsets = coordinator.fetchCommittedOffsets(Collections.singleton(partition));
+                committed = offsets.get(partition);
+            }
+
+            if (committed == null)
+                throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition);
+
+            return committed;
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Get the metrics kept by the consumer
+     */
+    @Override
+    public Map<MetricName, ? extends Metric> metrics() {
+        return Collections.unmodifiableMap(this.metrics.metrics());
+    }
+
+    /**
+     * Get metadata about the partitions for a given topic. This method will issue a remote call to the server if it
+     * does not already have any metadata about the given topic.
+     * 
+     * @param topic The topic to get partition metadata for
+     * @return The list of partitions
+     */
+    @Override
+    public List<PartitionInfo> partitionsFor(String topic) {
+        acquire();
+        try {
+            Cluster cluster = this.metadata.fetch();
+            List<PartitionInfo> parts = cluster.partitionsForTopic(topic);
+            if (parts == null) {
+                metadata.add(topic);
+                client.awaitMetadataUpdate();
+                parts = metadata.fetch().partitionsForTopic(topic);
+            }
+            return parts;
+        } finally {
+            release();
+        }
+    }
+
+    @Override
+    public void close() {
+        acquire();
+        try {
+            if (closed) return;
+            close(false);
+        } finally {
+            release();
+        }
+    }
+
+    /**
+     * Wakeup the consumer. This method is thread-safe and is useful in particular to abort a long poll.
+     * The thread which is blocking in an operation will throw {@link ConsumerWakeupException}.
+     */
+    @Override
+    public void wakeup() {
+        this.client.wakeup();
+    }
+
+    private void close(boolean swallowException) {
+        log.trace("Closing the Kafka consumer.");
+        AtomicReference<Throwable> firstException = new AtomicReference<Throwable>();
+        this.closed = true;
+        ClientUtils.closeQuietly(metrics, "consumer metrics", firstException);
+        ClientUtils.closeQuietly(client, "consumer network client", firstException);
+        ClientUtils.closeQuietly(keyDeserializer, "consumer key deserializer", firstException);
+        ClientUtils.closeQuietly(valueDeserializer, "consumer value deserializer", firstException);
+        log.debug("The Kafka consumer has closed.");
+        if (firstException.get() != null && !swallowException) {
+            throw new KafkaException("Failed to close kafka consumer", firstException.get());
+        }
+    }
+
+    private Coordinator.RebalanceCallback wrapRebalanceCallback(final ConsumerRebalanceCallback callback) {
+        return new Coordinator.RebalanceCallback() {
+            @Override
+            public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+                callback.onPartitionsAssigned(KafkaConsumer.this, partitions);
+            }
+
+            @Override
+            public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+                callback.onPartitionsRevoked(KafkaConsumer.this, partitions);
+            }
+        };
+    }
+
+    /**
+     * Set the fetch position to the committed position (if there is one)
+     * or reset it using the offset reset policy the user has configured.
+     *
+     * @param partitions The partitions that needs updating fetch positions
+     * @throws NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is
+     *             defined
+     */
+    private void updateFetchPositions(Set<TopicPartition> partitions) {
+        // refresh commits for all assigned partitions
+        coordinator.refreshCommittedOffsetsIfNeeded();
+
+        // then do any offset lookups in case some positions are not known
+        fetcher.updateFetchPositions(partitions);
+    }
+
+    /*
+     * Check that the consumer hasn't been closed.
+     */
+    private void ensureNotClosed() {
+        if (this.closed)
+            throw new IllegalStateException("This consumer has already been closed.");
+    }
+
+    /**
+     * Acquire the light lock protecting this consumer from multi-threaded access. Instead of blocking
+     * when the lock is not available, however, we just throw an exception (since multi-threaded usage is not
+     * supported).
+     * @throws IllegalStateException if the consumer has been closed
+     * @throws ConcurrentModificationException if another thread already has the lock
+     */
+    private void acquire() {
+        ensureNotClosed();
+        long threadId = Thread.currentThread().getId();
+        if (threadId != currentThread.get() && !currentThread.compareAndSet(NO_CURRENT_THREAD, threadId))
+            throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access");
+        refcount.incrementAndGet();
+    }
+
+    /**
+     * Release the light lock protecting the consumer from multi-threaded access.
+     */
+    private void release() {
+        if (refcount.decrementAndGet() == 0)
+            currentThread.set(NO_CURRENT_THREAD);
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/MockConsumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/MockConsumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/MockConsumer.java
new file mode 100644
index 0000000..1d08519
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/MockConsumer.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer;
+
+import org.apache.flink.kafka_backport.clients.consumer.internals.SubscriptionState;
+import org.apache.flink.kafka_backport.common.Metric;
+import org.apache.flink.kafka_backport.common.MetricName;
+import org.apache.flink.kafka_backport.common.PartitionInfo;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. This class is <i> not
+ * threadsafe </i>
+ * <p>
+ * The consumer runs in the user thread and multiplexes I/O over TCP connections to each of the brokers it needs to
+ * communicate with. Failure to close the consumer after use will leak these resources.
+ */
+public class MockConsumer<K, V> implements Consumer<K, V> {
+
+    private final Map<String, List<PartitionInfo>> partitions;
+    private final SubscriptionState subscriptions;
+    private Map<TopicPartition, List<ConsumerRecord<K, V>>> records;
+    private boolean closed;
+
+    public MockConsumer(OffsetResetStrategy offsetResetStrategy) {
+        this.subscriptions = new SubscriptionState(offsetResetStrategy);
+        this.partitions = new HashMap<String, List<PartitionInfo>>();
+        this.records = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
+        this.closed = false;
+    }
+    
+    @Override
+    public synchronized Set<TopicPartition> subscriptions() {
+        return this.subscriptions.assignedPartitions();
+    }
+
+    @Override
+    public synchronized void subscribe(String... topics) {
+        ensureNotClosed();
+        for (String topic : topics)
+            this.subscriptions.subscribe(topic);
+    }
+
+    @Override
+    public synchronized void subscribe(TopicPartition... partitions) {
+        ensureNotClosed();
+        for (TopicPartition partition : partitions)
+            this.subscriptions.subscribe(partition);
+    }
+
+    public synchronized void unsubscribe(String... topics) {
+        ensureNotClosed();
+        for (String topic : topics)
+            this.subscriptions.unsubscribe(topic);
+    }
+
+    public synchronized void unsubscribe(TopicPartition... partitions) {
+        ensureNotClosed();
+        for (TopicPartition partition : partitions)
+            this.subscriptions.unsubscribe(partition);
+    }
+
+    @Override
+    public synchronized ConsumerRecords<K, V> poll(long timeout) {
+        ensureNotClosed();
+        // update the consumed offset
+        for (Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : this.records.entrySet()) {
+            List<ConsumerRecord<K, V>> recs = entry.getValue();
+            if (!recs.isEmpty())
+                this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset());
+        }
+
+        ConsumerRecords<K, V> copy = new ConsumerRecords<K, V>(this.records);
+        this.records = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
+        return copy;
+    }
+
+    public synchronized void addRecord(ConsumerRecord<K, V> record) {
+        ensureNotClosed();
+        TopicPartition tp = new TopicPartition(record.topic(), record.partition());
+        this.subscriptions.assignedPartitions().add(tp);
+        List<ConsumerRecord<K, V>> recs = this.records.get(tp);
+        if (recs == null) {
+            recs = new ArrayList<ConsumerRecord<K, V>>();
+            this.records.put(tp, recs);
+        }
+        recs.add(record);
+    }
+
+    @Override
+    public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
+        ensureNotClosed();
+        for (Entry<TopicPartition, Long> entry : offsets.entrySet())
+            subscriptions.committed(entry.getKey(), entry.getValue());
+        if (callback != null) {
+            callback.onComplete(offsets, null);
+        }
+    }
+
+    @Override
+    public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType) {
+        commit(offsets, commitType, null);
+    }
+
+    @Override
+    public synchronized void commit(CommitType commitType, ConsumerCommitCallback callback) {
+        ensureNotClosed();
+        commit(this.subscriptions.allConsumed(), commitType, callback);
+    }
+
+    @Override
+    public synchronized void commit(CommitType commitType) {
+        commit(commitType, null);
+    }
+
+    @Override
+    public synchronized void seek(TopicPartition partition, long offset) {
+        ensureNotClosed();
+        subscriptions.seek(partition, offset);
+    }
+
+    @Override
+    public synchronized long committed(TopicPartition partition) {
+        ensureNotClosed();
+        return subscriptions.committed(partition);
+    }
+
+    @Override
+    public synchronized long position(TopicPartition partition) {
+        ensureNotClosed();
+        return subscriptions.consumed(partition);
+    }
+
+    @Override
+    public synchronized void seekToBeginning(TopicPartition... partitions) {
+        ensureNotClosed();
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public synchronized void seekToEnd(TopicPartition... partitions) {
+        ensureNotClosed();
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Map<MetricName, ? extends Metric> metrics() {
+        ensureNotClosed();
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public synchronized List<PartitionInfo> partitionsFor(String topic) {
+        ensureNotClosed();
+        List<PartitionInfo> parts = this.partitions.get(topic);
+        if (parts == null)
+            return Collections.emptyList();
+        else
+            return parts;
+    }
+
+    public synchronized void updatePartitions(String topic, List<PartitionInfo> partitions) {
+        ensureNotClosed();
+        this.partitions.put(topic, partitions);
+    }
+
+    @Override
+    public synchronized void close() {
+        ensureNotClosed();
+        this.closed = true;
+    }
+
+    @Override
+    public void wakeup() {
+
+    }
+
+    private void ensureNotClosed() {
+        if (this.closed)
+            throw new IllegalStateException("This consumer has already been closed.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/NoOffsetForPartitionException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/NoOffsetForPartitionException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/NoOffsetForPartitionException.java
new file mode 100644
index 0000000..19ae0a6
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/NoOffsetForPartitionException.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.kafka_backport.clients.consumer;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Indicates that there is no stored offset and no defined offset reset policy
+ */
+public class NoOffsetForPartitionException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public NoOffsetForPartitionException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/OffsetResetStrategy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/OffsetResetStrategy.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/OffsetResetStrategy.java
new file mode 100644
index 0000000..70c254f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/OffsetResetStrategy.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public enum OffsetResetStrategy {
+    LATEST, EARLIEST, NONE
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/ConsumerNetworkClient.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/ConsumerNetworkClient.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/ConsumerNetworkClient.java
new file mode 100644
index 0000000..a6d16cd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/consumer/internals/ConsumerNetworkClient.java
@@ -0,0 +1,296 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients.consumer.internals;
+
+import org.apache.flink.kafka_backport.clients.ClientResponse;
+import org.apache.flink.kafka_backport.clients.Metadata;
+import org.apache.flink.kafka_backport.clients.consumer.ConsumerWakeupException;
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.requests.AbstractRequest;
+import org.apache.flink.kafka_backport.common.requests.RequestHeader;
+import org.apache.flink.kafka_backport.common.requests.RequestSend;
+import org.apache.flink.kafka_backport.clients.ClientRequest;
+import org.apache.flink.kafka_backport.clients.KafkaClient;
+import org.apache.flink.kafka_backport.clients.RequestCompletionHandler;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.utils.Time;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Higher level consumer access to the network layer with basic support for futures and
+ * task scheduling. NOT thread-safe!
+ *
+ * TODO: The current implementation is simplistic in that it provides a facility for queueing requests
+ * prior to delivery, but it makes no effort to retry requests which cannot be sent at the time
+ * {@link #poll(long)} is called. This makes the behavior of the queue predictable and easy to
+ * understand, but there are opportunities to provide timeout or retry capabilities in the future.
+ * How we do this may depend on KAFKA-2120, so for now, we retain the simplistic behavior.
+ */
+public class ConsumerNetworkClient implements Closeable {
+    private final KafkaClient client;
+    private final AtomicBoolean wakeup = new AtomicBoolean(false);
+    private final DelayedTaskQueue delayedTasks = new DelayedTaskQueue();
+    private final Map<Node, List<ClientRequest>> unsent = new HashMap<Node, List<ClientRequest>>();
+    private final Metadata metadata;
+    private final Time time;
+    private final long retryBackoffMs;
+
+    public ConsumerNetworkClient(KafkaClient client,
+                                 Metadata metadata,
+                                 Time time,
+                                 long retryBackoffMs) {
+        this.client = client;
+        this.metadata = metadata;
+        this.time = time;
+        this.retryBackoffMs = retryBackoffMs;
+    }
+
+    /**
+     * Schedule a new task to be executed at the given time. This is "best-effort" scheduling and
+     * should only be used for coarse synchronization.
+     * @param task The task to be scheduled
+     * @param at The time it should run
+     */
+    public void schedule(DelayedTask task, long at) {
+        delayedTasks.add(task, at);
+    }
+
+    /**
+     * Unschedule a task. This will remove all instances of the task from the task queue.
+     * This is a no-op if the task is not scheduled.
+     * @param task The task to be unscheduled.
+     */
+    public void unschedule(DelayedTask task) {
+        delayedTasks.remove(task);
+    }
+
+    /**
+     * Send a new request. Note that the request is not actually transmitted on the
+     * network until one of the {@link #poll(long)} variants is invoked. At this
+     * point the request will either be transmitted successfully or will fail.
+     * Use the returned future to obtain the result of the send.
+     * @param node The destination of the request
+     * @param api The Kafka API call
+     * @param request The request payload
+     * @return A future which indicates the result of the send.
+     */
+    public RequestFuture<ClientResponse> send(Node node,
+                                              ApiKeys api,
+                                              AbstractRequest request) {
+        long now = time.milliseconds();
+        RequestFutureCompletionHandler future = new RequestFutureCompletionHandler();
+        RequestHeader header = client.nextRequestHeader(api);
+        RequestSend send = new RequestSend(node.idString(), header, request.toStruct());
+        put(node, new ClientRequest(now, true, send, future));
+        return future;
+    }
+
+    private void put(Node node, ClientRequest request) {
+        List<ClientRequest> nodeUnsent = unsent.get(node);
+        if (nodeUnsent == null) {
+            nodeUnsent = new ArrayList<ClientRequest>();
+            unsent.put(node, nodeUnsent);
+        }
+        nodeUnsent.add(request);
+    }
+
+    public Node leastLoadedNode() {
+        return client.leastLoadedNode(time.milliseconds());
+    }
+
+    /**
+     * Block until the metadata has been refreshed.
+     */
+    public void awaitMetadataUpdate() {
+        int version = this.metadata.requestUpdate();
+        do {
+            poll(Long.MAX_VALUE);
+        } while (this.metadata.version() == version);
+    }
+
+    /**
+     * Wakeup an active poll. This will cause the polling thread to throw an exception either
+     * on the current poll if one is active, or the next poll.
+     */
+    public void wakeup() {
+        this.wakeup.set(true);
+        this.client.wakeup();
+    }
+
+    /**
+     * Block indefinitely until the given request future has finished.
+     * @param future The request future to await.
+     * @throws org.apache.flink.kafka_backport.clients.consumer.ConsumerWakeupException if {@link #wakeup()} is called from another thread
+     */
+    public void poll(RequestFuture<?> future) {
+        while (!future.isDone())
+            poll(Long.MAX_VALUE);
+    }
+
+    /**
+     * Block until the provided request future request has finished or the timeout has expired.
+     * @param future The request future to wait for
+     * @param timeout The maximum duration (in ms) to wait for the request
+     * @return true if the future is done, false otherwise
+     * @throws org.apache.flink.kafka_backport.clients.consumer.ConsumerWakeupException if {@link #wakeup()} is called from another thread
+     */
+    public boolean poll(RequestFuture<?> future, long timeout) {
+        long now = time.milliseconds();
+        long deadline = now + timeout;
+        while (!future.isDone() && now < deadline) {
+            poll(deadline - now, now);
+            now = time.milliseconds();
+        }
+        return future.isDone();
+    }
+
+    /**
+     * Poll for any network IO. All send requests will either be transmitted on the network
+     * or failed when this call completes.
+     * @param timeout The maximum time to wait for an IO event.
+     * @throws org.apache.flink.kafka_backport.clients.consumer.ConsumerWakeupException if {@link #wakeup()} is called from another thread
+     */
+    public void poll(long timeout) {
+        poll(timeout, time.milliseconds());
+    }
+
+    private void poll(long timeout, long now) {
+        // send all the requests we can send now
+        pollUnsentRequests(now);
+
+        // ensure we don't poll any longer than the deadline for
+        // the next scheduled task
+        timeout = Math.min(timeout, delayedTasks.nextTimeout(now));
+        clientPoll(timeout, now);
+
+        // execute scheduled tasks
+        now = time.milliseconds();
+        delayedTasks.poll(now);
+
+        // try again to send requests since buffer space may have been
+        // cleared or a connect finished in the poll
+        pollUnsentRequests(now);
+
+        // fail all requests that couldn't be sent
+        clearUnsentRequests(now);
+
+    }
+
+    /**
+     * Block until all pending requests from the given node have finished.
+     * @param node The node to await requests from
+     */
+    public void awaitPendingRequests(Node node) {
+        while (pendingRequestCount(node) > 0)
+            poll(retryBackoffMs);
+    }
+
+    /**
+     * Get the count of pending requests to the given node. This includes both request that
+     * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission.
+     * @param node The node in question
+     * @return The number of pending requests
+     */
+    public int pendingRequestCount(Node node) {
+        List<ClientRequest> pending = unsent.get(node);
+        int unsentCount = pending == null ? 0 : pending.size();
+        return unsentCount + client.inFlightRequestCount(node.idString());
+    }
+
+    /**
+     * Get the total count of pending requests from all nodes. This includes both requests that
+     * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission.
+     * @return The total count of pending requests
+     */
+    public int pendingRequestCount() {
+        int total = 0;
+        for (List<ClientRequest> requests: unsent.values())
+            total += requests.size();
+        return total + client.inFlightRequestCount();
+    }
+
+    private void pollUnsentRequests(long now) {
+        while (trySend(now))
+            clientPoll(0, now);
+    }
+
+    private void clearUnsentRequests(long now) {
+        // clear all unsent requests and fail their corresponding futures
+        for (Map.Entry<Node, List<ClientRequest>> requestEntry: unsent.entrySet()) {
+            Iterator<ClientRequest> iterator = requestEntry.getValue().iterator();
+            while (iterator.hasNext()) {
+                ClientRequest request = iterator.next();
+                RequestFutureCompletionHandler handler =
+                        (RequestFutureCompletionHandler) request.callback();
+                handler.raise(SendFailedException.INSTANCE);
+                iterator.remove();
+            }
+        }
+        unsent.clear();
+    }
+
+    private boolean trySend(long now) {
+        // send any requests that can be sent now
+        boolean requestsSent = false;
+        for (Map.Entry<Node, List<ClientRequest>> requestEntry: unsent.entrySet()) {
+            Node node = requestEntry.getKey();
+            Iterator<ClientRequest> iterator = requestEntry.getValue().iterator();
+            while (iterator.hasNext()) {
+                ClientRequest request = iterator.next();
+                if (client.ready(node, now)) {
+                    client.send(request);
+                    iterator.remove();
+                    requestsSent = true;
+                } else if (client.connectionFailed(node)) {
+                    RequestFutureCompletionHandler handler =
+                            (RequestFutureCompletionHandler) request.callback();
+                    handler.onComplete(new ClientResponse(request, now, true, null));
+                    iterator.remove();
+                }
+            }
+        }
+        return requestsSent;
+    }
+
+    private void clientPoll(long timeout, long now) {
+        client.poll(timeout, now);
+        if (wakeup.get()) {
+            clearUnsentRequests(now);
+            wakeup.set(false);
+            throw new ConsumerWakeupException();
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        client.close();
+    }
+
+    public static class RequestFutureCompletionHandler
+            extends RequestFuture<ClientResponse>
+            implements RequestCompletionHandler {
+
+        @Override
+        public void onComplete(ClientResponse response) {
+            complete(response);
+        }
+    }
+}


[32/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
deleted file mode 100644
index 8c1883e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
+++ /dev/null
@@ -1,1225 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either 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.assertEquals;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.UUID;
-
-import kafka.admin.AdminUtils;
-import kafka.api.PartitionMetadata;
-import kafka.consumer.Consumer;
-import kafka.consumer.ConsumerConfig;
-import kafka.consumer.ConsumerIterator;
-import kafka.consumer.KafkaStream;
-import kafka.javaapi.consumer.ConsumerConnector;
-import kafka.message.MessageAndMetadata;
-import kafka.network.SocketServer;
-import kafka.server.KafkaConfig;
-import kafka.server.KafkaServer;
-
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.commons.collections.map.LinkedMap;
-import org.apache.curator.test.TestingServer;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.MapFunction;
-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.TypeInfoParser;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.net.NetUtils;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kafka.api.KafkaSink;
-import org.apache.flink.streaming.connectors.kafka.api.KafkaSource;
-import org.apache.flink.streaming.connectors.kafka.api.persistent.PersistentKafkaSource;
-import org.apache.flink.streaming.connectors.kafka.partitioner.SerializableKafkaPartitioner;
-import org.apache.flink.streaming.connectors.kafka.util.KafkaLocalSystemTime;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.util.Collector;
-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 scala.collection.Seq;
-
-/**
- * Code in this test is based on the following GitHub repository:
- * (as per commit bc6b2b2d5f6424d5f377aa6c0871e82a956462ef)
- * <p/>
- * https://github.com/sakserv/hadoop-mini-clusters (ASL licensed)
- */
-@SuppressWarnings("serial")
-public class KafkaITCase {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KafkaITCase.class);
-	private static final int NUMBER_OF_KAFKA_SERVERS = 3;
-
-	private static int zkPort;
-	private static String kafkaHost;
-
-	private static String zookeeperConnectionString;
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-	public static File tmpZkDir;
-	public static List<File> tmpKafkaDirs;
-
-	private static TestingServer zookeeper;
-	private static List<KafkaServer> brokers;
-	private static String brokerConnectionStrings = "";
-
-	private static ConsumerConfig standardCC;
-
-	private static ZkClient zkClient;
-
-
-	@BeforeClass
-	public static void prepare() throws IOException {
-		LOG.info("Starting KafkaITCase.prepare()");
-		tmpZkDir = tempFolder.newFolder();
-
-		tmpKafkaDirs = new ArrayList<File>(NUMBER_OF_KAFKA_SERVERS);
-		for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
-			tmpKafkaDirs.add(tempFolder.newFolder());
-		}
-
-		kafkaHost = InetAddress.getLocalHost().getHostName();
-		zkPort = NetUtils.getAvailablePort();
-		zookeeperConnectionString = "localhost:" + zkPort;
-
-		zookeeper = null;
-		brokers = null;
-
-		try {
-			LOG.info("Starting Zookeeper");
-			zookeeper = getZookeeper();
-			LOG.info("Starting KafkaServer");
-			brokers = new ArrayList<KafkaServer>(NUMBER_OF_KAFKA_SERVERS);
-			for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
-				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i)));
-				SocketServer socketServer = brokers.get(i).socketServer();
-				String host = "localhost";
-				if(socketServer.host() != null) {
-					host = socketServer.host();
-				}
-				brokerConnectionStrings += host+":"+socketServer.port()+",";
-			}
-
-			LOG.info("ZK and KafkaServer started.");
-		} catch (Throwable t) {
-			LOG.warn("Test failed with exception", t);
-			Assert.fail("Test failed with: " + t.getMessage());
-		}
-
-		Properties cProps = new Properties();
-		cProps.setProperty("zookeeper.connect", zookeeperConnectionString);
-		cProps.setProperty("group.id", "flink-tests");
-		cProps.setProperty("auto.commit.enable", "false");
-
-		cProps.setProperty("auto.offset.reset", "smallest"); // read from the beginning.
-
-		standardCC = new ConsumerConfig(cProps);
-
-		zkClient = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), standardCC.zkConnectionTimeoutMs(), new PersistentKafkaSource.KafkaZKStringSerializer());
-	}
-
-	@AfterClass
-	public static void shutDownServices() {
-		LOG.info("Shutting down all services");
-		for (KafkaServer broker : brokers) {
-			if (broker != null) {
-				broker.shutdown();
-			}
-		}
-		if (zookeeper != null) {
-			try {
-				zookeeper.stop();
-			} catch (IOException e) {
-				LOG.warn("ZK.stop() failed", e);
-			}
-		}
-		zkClient.close();
-	}
-
-	// --------------------------  test checkpointing ------------------------
-	@Test
-	public void testCheckpointing() throws Exception {
-		createTestTopic("testCheckpointing", 1, 1);
-
-		Properties props = new Properties();
-		props.setProperty("zookeeper.connect", zookeeperConnectionString);
-		props.setProperty("group.id", "testCheckpointing");
-		props.setProperty("auto.commit.enable", "false");
-		ConsumerConfig cc = new ConsumerConfig(props);
-		PersistentKafkaSource<String> source = new PersistentKafkaSource<String>("testCheckpointing", new FakeDeserializationSchema(), cc);
-
-
-		Field pendingCheckpointsField = PersistentKafkaSource.class.getDeclaredField("pendingCheckpoints");
-		pendingCheckpointsField.setAccessible(true);
-		LinkedMap pendingCheckpoints = (LinkedMap) pendingCheckpointsField.get(source);
-
-
-		Assert.assertEquals(0, pendingCheckpoints.size());
-		// first restore
-		source.restoreState(new long[]{1337});
-		// then open
-		source.open(new Configuration());
-		long[] state1 = source.snapshotState(1, 15);
-		Assert.assertArrayEquals(new long[]{1337}, state1);
-		long[] state2 = source.snapshotState(2, 30);
-		Assert.assertArrayEquals(new long[]{1337}, state2);
-		Assert.assertEquals(2, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(1);
-		Assert.assertEquals(1, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(2);
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(666); // invalid checkpoint
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		// create 500 snapshots
-		for(int i = 0; i < 500; i++) {
-			source.snapshotState(i, 15 * i);
-		}
-		Assert.assertEquals(500, pendingCheckpoints.size());
-
-		// commit only the second last
-		source.notifyCheckpointComplete(498);
-		Assert.assertEquals(1, pendingCheckpoints.size());
-
-		// access invalid checkpoint
-		source.notifyCheckpointComplete(490);
-
-		// and the last
-		source.notifyCheckpointComplete(499);
-		Assert.assertEquals(0, pendingCheckpoints.size());
-	}
-
-
-	private static class FakeDeserializationSchema implements DeserializationSchema<String> {
-
-		@Override
-		public String deserialize(byte[] message) {
-			return null;
-		}
-
-		@Override
-		public boolean isEndOfStream(String nextElement) {
-			return false;
-		}
-
-		@Override
-		public TypeInformation<String> getProducedType() {
-			return null;
-		}
-	}
-
-	// ---------------------------------------------------------------
-
-
-	@Test
-	public void testOffsetManipulation() {
-		ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), standardCC.zkConnectionTimeoutMs(), new PersistentKafkaSource.KafkaZKStringSerializer());
-
-		final String topicName = "testOffsetManipulation";
-
-		// create topic
-		Properties topicConfig = new Properties();
-		LOG.info("Creating topic {}", topicName);
-		AdminUtils.createTopic(zk, topicName, 3, 2, topicConfig);
-
-		PersistentKafkaSource.setOffset(zk, standardCC.groupId(), topicName, 0, 1337);
-
-		Assert.assertEquals(1337L, PersistentKafkaSource.getOffset(zk, standardCC.groupId(), topicName, 0));
-
-		zk.close();
-	}
-
-	public static class TestPersistentKafkaSource<OUT> extends PersistentKafkaSource<OUT> {
-		private static Object sync = new Object();
-		public static long[] finalOffset;
-		public TestPersistentKafkaSource(String topicName, DeserializationSchema<OUT> deserializationSchema, ConsumerConfig consumerConfig) {
-			super(topicName, deserializationSchema, consumerConfig);
-		}
-
-		@Override
-		public void close() {
-			super.close();
-			LOG.info("Starting close " +Arrays.toString(commitedOffsets));
-			synchronized (sync) {
-				if (finalOffset == null) {
-					finalOffset = new long[commitedOffsets.length];
-				}
-				for(int i = 0; i < commitedOffsets.length; i++) {
-					if(commitedOffsets[i] > 0) {
-						if(finalOffset[i] > 0) {
-							throw new RuntimeException("This is unexpected on i = "+i);
-						}
-						finalOffset[i] = commitedOffsets[i];
-					}
-				}
-			}
-			LOG.info("Finished closing. Final "+Arrays.toString(finalOffset));
-		}
-	}
-	/**
-	 * We want to use the High level java consumer API but manage the offset in Zookeeper manually.
-	 *
-	 */
-	@Test
-	public void testPersistentSourceWithOffsetUpdates() throws Exception {
-		LOG.info("Starting testPersistentSourceWithOffsetUpdates()");
-
-		ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), standardCC.zkConnectionTimeoutMs(), new PersistentKafkaSource.KafkaZKStringSerializer());
-
-		final String topicName = "testOffsetHacking";
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(3);
-		env.getConfig().disableSysoutLogging();
-		env.enableCheckpointing(50);
-		env.setNumberOfExecutionRetries(0);
-
-		// create topic
-		Properties topicConfig = new Properties();
-		LOG.info("Creating topic {}", topicName);
-		AdminUtils.createTopic(zk, topicName, 3, 2, topicConfig);
-
-		// write a sequence from 0 to 99 to each of the three partitions.
-		writeSequence(env, topicName, 0, 99);
-
-		readSequence(env, standardCC, topicName, 0, 100, 300);
-
-		LOG.info("State in persistent kafka sources {}", TestPersistentKafkaSource.finalOffset);
-
-		// check offsets to be set at least higher than 50.
-		// correctly, we would expect them to be set to 99, but right now there is no way of stopping a topology once all pending
-		// checkpoints have been committed.
-		// To work around that limitation, the persistent kafka consumer is throtteled with a thread.sleep().
-
-		long o1 = -1, o2 = -1, o3 = -1;
-		if(TestPersistentKafkaSource.finalOffset[0] > 0) {
-			o1 = PersistentKafkaSource.getOffset(zk, standardCC.groupId(), topicName, 0);
-			Assert.assertTrue("The offset seems incorrect, got " + o1, o1 == TestPersistentKafkaSource.finalOffset[0]);
-		}
-		if(TestPersistentKafkaSource.finalOffset[1] > 0) {
-			o2 = PersistentKafkaSource.getOffset(zk, standardCC.groupId(), topicName, 1);
-			Assert.assertTrue("The offset seems incorrect, got " + o2, o2 == TestPersistentKafkaSource.finalOffset[1]);
-		}
-		if(TestPersistentKafkaSource.finalOffset[2] > 0) {
-			o3 = PersistentKafkaSource.getOffset(zk, standardCC.groupId(), topicName, 2);
-			Assert.assertTrue("The offset seems incorrect, got " + o3, o3 == TestPersistentKafkaSource.finalOffset[2]);
-		}
-		Assert.assertFalse("no offset has been set", TestPersistentKafkaSource.finalOffset[0] == 0 &&
-													TestPersistentKafkaSource.finalOffset[1] == 0 &&
-													TestPersistentKafkaSource.finalOffset[2] == 0);
-		LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
-
-		LOG.info("Manipulating offsets");
-		// set the offset to 50 for the three partitions
-		PersistentKafkaSource.setOffset(zk, standardCC.groupId(), topicName, 0, 50);
-		PersistentKafkaSource.setOffset(zk, standardCC.groupId(), topicName, 1, 50);
-		PersistentKafkaSource.setOffset(zk, standardCC.groupId(), topicName, 2, 50);
-
-		// create new env
-		env = StreamExecutionEnvironment.createLocalEnvironment(3);
-		env.getConfig().disableSysoutLogging();
-		readSequence(env, standardCC, topicName, 50, 50, 150);
-
-		zk.close();
-
-		LOG.info("Finished testPersistentSourceWithOffsetUpdates()");
-	}
-
-	private void readSequence(StreamExecutionEnvironment env, ConsumerConfig cc, final String topicName, final int valuesStartFrom, final int valuesCount, final int finalCount) throws Exception {
-		LOG.info("Reading sequence for verification until final count {}", finalCount);
-
-		TypeInformation<Tuple2<Integer, Integer>> tuple2info = TypeInfoParser.parse("Tuple2<Integer, Integer>");
-		
-		TestPersistentKafkaSource<Tuple2<Integer, Integer>> pks = new TestPersistentKafkaSource<>(topicName, 
-				new TypeInformationSerializationSchema<>(tuple2info, env.getConfig()), cc);
-		
-		DataStream<Tuple2<Integer, Integer>> source = env.addSource(pks).map(new MapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>>() {
-			// we need to slow down the source so that it can participate in a few checkpoints.
-			// Otherwise it would write its data into buffers and shut down.
-			@Override
-			public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) throws Exception {
-				Thread.sleep(50);
-				return value;
-			}
-		});
-
-		// verify data
-		DataStream<Integer> validIndexes = source.flatMap(new RichFlatMapFunction<Tuple2<Integer, Integer>, Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			int[] values = new int[valuesCount];
-			int count = 0;
-
-			@Override
-			public void flatMap(Tuple2<Integer, Integer> value, Collector<Integer> out) throws Exception {
-				values[value.f1 - valuesStartFrom]++;
-				count++;
-
-				LOG.info("Reader " + getRuntimeContext().getIndexOfThisSubtask() + " got " + value + " count=" + count + "/" + finalCount);
-				// verify if we've seen everything
-				if (count == finalCount) {
-					LOG.info("Received all values");
-					for (int i = 0; i < values.length; i++) {
-						int v = values[i];
-						if (v != 3) {
-							LOG.warn("Test is going to fail");
-							printTopic(topicName, valuesCount, this.getRuntimeContext().getExecutionConfig());
-							throw new RuntimeException("Expected v to be 3, but was " + v + " on element " + i + " array=" + Arrays.toString(values));
-						}
-					}
-					// test has passed
-					throw new SuccessException();
-				}
-			}
-
-		}).setParallelism(1);
-
-		tryExecute(env, "Read data from Kafka");
-
-		LOG.info("Successfully read sequence for verification");
-	}
-
-
-
-	private void writeSequence(StreamExecutionEnvironment env, String topicName, final int from, final int to) throws Exception {
-		LOG.info("Writing sequence from {} to {} to topic {}", from, to, topicName);
-
-		TypeInformation<Tuple2<Integer, Integer>> tuple2info = TypeInfoParser.parse("Tuple2<Integer, Integer>");
-		
-		DataStream<Tuple2<Integer, Integer>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
-			private static final long serialVersionUID = 1L;
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = from;
-				int partition = getRuntimeContext().getIndexOfThisSubtask();
-				while (running) {
-					LOG.info("Writing " + cnt + " to partition " + partition);
-					ctx.collect(new Tuple2<Integer, Integer>(getRuntimeContext().getIndexOfThisSubtask(),
-							cnt));
-					if (cnt == to) {
-						LOG.info("Writer reached end.");
-						return;
-					}
-					cnt++;
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		}).setParallelism(3);
-		stream.addSink(new KafkaSink<Tuple2<Integer, Integer>>(brokerConnectionStrings,
-				topicName,
-				new TypeInformationSerializationSchema<>(tuple2info, env.getConfig()),
-				new T2Partitioner()
-		)).setParallelism(3);
-		env.execute("Write sequence from " + from + " to " + to + " to topic " + topicName);
-		LOG.info("Finished writing sequence");
-	}
-
-	private static class T2Partitioner implements SerializableKafkaPartitioner {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public int partition(Object key, int numPartitions) {
-			if(numPartitions != 3) {
-				throw new IllegalArgumentException("Expected three partitions");
-			}
-			
-			@SuppressWarnings("unchecked")
-			Tuple2<Integer, Integer> element = (Tuple2<Integer, Integer>) key;
-			return element.f0;
-		}
-	}
-
-
-	@Test
-	public void regularKafkaSourceTest() throws Exception {
-		LOG.info("Starting KafkaITCase.regularKafkaSourceTest()");
-
-		String topic = "regularKafkaSourceTestTopic";
-		createTestTopic(topic, 1, 1);
-
-		TypeInformation<Tuple2<Long, String>> longStringInfo = TypeInfoParser.parse("Tuple2<Long, String>");
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-		// add consuming topology:
-		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(
-				new KafkaSource<Tuple2<Long, String>>(zookeeperConnectionString, topic, "myFlinkGroup",
-						new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig()), 5000));
-		
-		consuming.addSink(new SinkFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-
-			int elCnt = 0;
-			int start = -1;
-			BitSet validator = new BitSet(101);
-
-			@Override
-			public void invoke(Tuple2<Long, String> value) throws Exception {
-				LOG.debug("Got value = " + value);
-				String[] sp = value.f1.split("-");
-				int v = Integer.parseInt(sp[1]);
-
-				assertEquals(value.f0 - 1000, (long) v);
-
-				if (start == -1) {
-					start = v;
-				}
-				Assert.assertFalse("Received tuple twice", validator.get(v - start));
-				validator.set(v - start);
-				elCnt++;
-				if (elCnt == 100) {
-					// check if everything in the bitset is set to true
-					int nc;
-					if ((nc = validator.nextClearBit(0)) != 100) {
-						throw new RuntimeException("The bitset was not set to 1 on all elements. Next clear:" + nc + " Set: " + validator);
-					}
-					throw new SuccessException();
-				}
-			}
-		});
-
-		// add producing topology
-		DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Long, String>> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = 0;
-				while (running) {
-					ctx.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++));
-					try {
-						Thread.sleep(100);
-					} catch (InterruptedException ignored) {
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		});
-		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, 
-				new TypeInformationSerializationSchema<Tuple2<Long, String>>(longStringInfo, env.getConfig())));
-
-		tryExecute(env, "regular kafka source test");
-
-		LOG.info("Finished KafkaITCase.regularKafkaSourceTest()");
-	}
-
-	@Test
-	public void tupleTestTopology() throws Exception {
-		LOG.info("Starting KafkaITCase.tupleTestTopology()");
-
-		String topic = "tupleTestTopic";
-		createTestTopic(topic, 1, 1);
-
-		TypeInformation<Tuple2<Long, String>> longStringInfo = TypeInfoParser.parse("Tuple2<Long, String>");
-		
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
-		// add consuming topology:
-		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(
-				new PersistentKafkaSource<>(topic,
-						new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig()),
-						standardCC
-				));
-		consuming.addSink(new RichSinkFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-
-			int elCnt = 0;
-			int start = -1;
-			BitSet validator = new BitSet(101);
-
-			@Override
-			public void invoke(Tuple2<Long, String> value) throws Exception {
-				LOG.info("Got value " + value);
-				String[] sp = value.f1.split("-");
-				int v = Integer.parseInt(sp[1]);
-
-				assertEquals(value.f0 - 1000, (long) v);
-
-				if (start == -1) {
-					start = v;
-				}
-				Assert.assertFalse("Received tuple twice", validator.get(v - start));
-				validator.set(v - start);
-				elCnt++;
-				if (elCnt == 100) {
-					// check if everything in the bitset is set to true
-					int nc;
-					if ((nc = validator.nextClearBit(0)) != 100) {
-						throw new RuntimeException("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();
-				Assert.assertTrue("No element received", elCnt > 0);
-			}
-		});
-
-		// add producing topology
-		DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Long, String>> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = 0;
-				while (running) {
-					ctx.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++));
-					LOG.info("Produced " + cnt);
-
-					try {
-						Thread.sleep(100);
-					} catch (InterruptedException ignored) {
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		});
-		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, 
-				new TypeInformationSerializationSchema<Tuple2<Long, String>>(longStringInfo, env.getConfig())));
-
-		tryExecute(env, "tupletesttopology");
-
-		LOG.info("Finished KafkaITCase.tupleTestTopology()");
-	}
-
-	/**
-	 * Test Flink's Kafka integration also with very big records (30MB)
-	 *
-	 * see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message
-	 *
-	 * @throws Exception
-	 */
-	@Test
-	public void bigRecordTestTopology() throws Exception {
-
-		LOG.info("Starting KafkaITCase.bigRecordTestTopology()");
-
-		String topic = "bigRecordTestTopic";
-		createTestTopic(topic, 1, 1);
-
-		final TypeInformation<Tuple2<Long, byte[]>> longBytesInfo = TypeInfoParser.parse("Tuple2<Long, byte[]>");
-		
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
-		// add consuming topology:
-		TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema = 
-				new TypeInformationSerializationSchema<Tuple2<Long, byte[]>>(longBytesInfo, env.getConfig());
-		
-		Properties consumerProps = new Properties();
-		consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 30));
-		consumerProps.setProperty("zookeeper.connect", zookeeperConnectionString);
-		consumerProps.setProperty("group.id", "test");
-		consumerProps.setProperty("auto.commit.enable", "false");
-		consumerProps.setProperty("auto.offset.reset", "smallest");
-
-		ConsumerConfig cc = new ConsumerConfig(consumerProps);
-		DataStreamSource<Tuple2<Long, byte[]>> consuming = env.addSource(
-				new PersistentKafkaSource<Tuple2<Long, byte[]>>(topic, serSchema, cc));
-
-		consuming.addSink(new SinkFunction<Tuple2<Long, byte[]>>() {
-			private static final long serialVersionUID = 1L;
-
-			int elCnt = 0;
-
-			@Override
-			public void invoke(Tuple2<Long, byte[]> value) throws Exception {
-				LOG.info("Received {}", value.f0);
-				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);
-				}
-			}
-		}).setParallelism(1);
-
-		// add producing topology
-		DataStream<Tuple2<Long, byte[]>> stream = env.addSource(new RichSourceFunction<Tuple2<Long, byte[]>>() {
-			private static final long serialVersionUID = 1L;
-			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 {
-				LOG.info("Starting source.");
-				long cnt = 0;
-				Random rnd = new Random(1337);
-				while (running) {
-					//
-					byte[] wl = new byte[Math.abs(rnd.nextInt(1024 * 1024 * 30))];
-					ctx.collect(new Tuple2<Long, byte[]>(cnt++, wl));
-					LOG.info("Emitted cnt=" + (cnt - 1) + " with byte.length = " + wl.length);
-
-					try {
-						Thread.sleep(100);
-					} catch (InterruptedException ignored) {
-					}
-					if(cnt == 10) {
-						LOG.info("Send end signal");
-						// signal end
-						ctx.collect(new Tuple2<Long, byte[]>(-1L, new byte[]{1}));
-						running = false;
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		});
-
-		stream.addSink(new KafkaSink<Tuple2<Long, byte[]>>(brokerConnectionStrings, topic,
-						new TypeInformationSerializationSchema<Tuple2<Long, byte[]>>(longBytesInfo, env.getConfig()))
-		);
-
-		tryExecute(env, "big topology test");
-
-		LOG.info("Finished KafkaITCase.bigRecordTestTopology()");
-	}
-
-
-	@Test
-	public void customPartitioningTestTopology() throws Exception {
-		LOG.info("Starting KafkaITCase.customPartitioningTestTopology()");
-
-		String topic = "customPartitioningTestTopic";
-
-		createTestTopic(topic, 3, 1);
-
-		final TypeInformation<Tuple2<Long, String>> longStringInfo = TypeInfoParser.parse("Tuple2<Long, String>");
-		
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
-		// add consuming topology:
-		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(
-				new PersistentKafkaSource<Tuple2<Long, String>>(topic,
-						new TypeInformationSerializationSchema<Tuple2<Long, String>>(longStringInfo, env.getConfig()),
-						standardCC));
-		consuming.addSink(new SinkFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-
-			int start = -1;
-			BitSet validator = new BitSet(101);
-
-			boolean gotPartition1 = false;
-			boolean gotPartition2 = false;
-			boolean gotPartition3 = false;
-
-			@Override
-			public void invoke(Tuple2<Long, String> value) throws Exception {
-				LOG.debug("Got " + value);
-				String[] sp = value.f1.split("-");
-				int v = Integer.parseInt(sp[1]);
-
-				assertEquals(value.f0 - 1000, (long) v);
-
-				switch (v) {
-					case 9:
-						gotPartition1 = true;
-						break;
-					case 19:
-						gotPartition2 = true;
-						break;
-					case 99:
-						gotPartition3 = true;
-						break;
-				}
-
-				if (start == -1) {
-					start = v;
-				}
-				Assert.assertFalse("Received tuple twice", validator.get(v - start));
-				validator.set(v - start);
-
-				if (gotPartition1 && gotPartition2 && gotPartition3) {
-					// check if everything in the bitset is set to true
-					int nc;
-					if ((nc = validator.nextClearBit(0)) != 100) {
-						throw new RuntimeException("The bitset was not set to 1 on all elements. Next clear:" + nc + " Set: " + validator);
-					}
-					throw new SuccessException();
-				}
-			}
-		});
-
-		// add producing topology
-		DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
-			private static final long serialVersionUID = 1L;
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Long, String>> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = 0;
-				while (running) {
-					ctx.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++));
-					try {
-						Thread.sleep(100);
-					} catch (InterruptedException ignored) {
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		});
-		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic,
-				new TypeInformationSerializationSchema<Tuple2<Long, String>>(longStringInfo, env.getConfig()), new CustomPartitioner()));
-
-		tryExecute(env, "custom partitioning test");
-
-		LOG.info("Finished KafkaITCase.customPartitioningTestTopology()");
-	}
-
-	/**
-	 * This is for a topic with 3 partitions and Tuple2<Long, String>
-	 */
-	private static class CustomPartitioner implements SerializableKafkaPartitioner {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public int partition(Object key, int numPartitions) {
-
-			@SuppressWarnings("unchecked")
-			Tuple2<Long, String> tuple = (Tuple2<Long, String>) key;
-			if (tuple.f0 < 10) {
-				return 0;
-			} else if (tuple.f0 < 20) {
-				return 1;
-			} else {
-				return 2;
-			}
-		}
-	}
-
-
-	@Test
-	public void simpleTestTopology() throws Exception {
-		String topic = "simpleTestTopic";
-
-		createTestTopic(topic, 1, 1);
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
-		// add consuming topology:
-		DataStreamSource<String> consuming = env.addSource(
-				new PersistentKafkaSource<String>(topic, new JavaDefaultStringSchema(), standardCC));
-		consuming.addSink(new SinkFunction<String>() {
-			private static final long serialVersionUID = 1L;
-
-			int elCnt = 0;
-			int start = -1;
-			BitSet validator = new BitSet(101);
-
-			@Override
-			public void invoke(String value) throws Exception {
-				LOG.debug("Got " + value);
-				String[] sp = value.split("-");
-				int v = Integer.parseInt(sp[1]);
-				if (start == -1) {
-					start = v;
-				}
-				Assert.assertFalse("Received tuple twice", validator.get(v - start));
-				validator.set(v - start);
-				elCnt++;
-				if (elCnt == 100) {
-					// check if everything in the bitset is set to true
-					int nc;
-					if ((nc = validator.nextClearBit(0)) != 100) {
-						throw new RuntimeException("The bitset was not set to 1 on all elements. Next clear:" + nc + " Set: " + validator);
-					}
-					throw new SuccessException();
-				}
-			}
-		});
-
-		// add producing topology
-		DataStream<String> stream = env.addSource(new SourceFunction<String>() {
-			private static final long serialVersionUID = 1L;
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<String> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = 0;
-				while (running) {
-					ctx.collect("kafka-" + cnt++);
-					try {
-						Thread.sleep(100);
-					} catch (InterruptedException ignored) {
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got cancel()");
-				running = false;
-			}
-		});
-		stream.addSink(new KafkaSink<String>(brokerConnectionStrings, topic, new JavaDefaultStringSchema()));
-
-		tryExecute(env, "simpletest");
-	}
-
-	private static boolean leaderHasShutDown = false;
-	private static boolean shutdownKafkaBroker;
-
-	@Test(timeout=60000)
-	public void brokerFailureTest() throws Exception {
-		String topic = "brokerFailureTestTopic";
-
-		createTestTopic(topic, 2, 2);
-
-		// --------------------------- write data to topic ---------------------
-		LOG.info("Writing data to topic {}", topic);
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
-		DataStream<String> stream = env.addSource(new SourceFunction<String>() {
-			private static final long serialVersionUID = 1L;
-
-			boolean running = true;
-
-			@Override
-			public void run(SourceContext<String> ctx) throws Exception {
-				LOG.info("Starting source.");
-				int cnt = 0;
-				while (running) {
-					String msg = "kafka-" + cnt++;
-					ctx.collect(msg);
-					LOG.info("sending message = "+msg);
-
-					if ((cnt - 1) % 20 == 0) {
-						LOG.debug("Sending message #{}", cnt - 1);
-					}
-					if(cnt == 200) {
-						LOG.info("Stopping to produce after 200 msgs");
-						break;
-					}
-
-				}
-			}
-
-			@Override
-			public void cancel() {
-				LOG.info("Source got chancel()");
-				running = false;
-			}
-		});
-		stream.addSink(new KafkaSink<String>(brokerConnectionStrings, topic, new JavaDefaultStringSchema()))
-				.setParallelism(1);
-
-		tryExecute(env, "broker failure test - writer");
-
-		// --------------------------- read and let broker fail ---------------------
-
-		LOG.info("Reading data from topic {} and let a broker fail", topic);
-		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);
-
-		final String leaderToShutDown = firstPart.leader().get().connectionString();
-		LOG.info("Leader to shutdown {}", leaderToShutDown);
-
-		final Thread brokerShutdown = new Thread(new Runnable() {
-			@Override
-			public void run() {
-				shutdownKafkaBroker = false;
-				while (!shutdownKafkaBroker) {
-					try {
-						Thread.sleep(10);
-					} catch (InterruptedException e) {
-						LOG.warn("Interruption", e);
-					}
-				}
-
-				for (KafkaServer kafkaServer : brokers) {
-					if (leaderToShutDown.equals(kafkaServer.config().advertisedHostName()+ ":"+ kafkaServer.config().advertisedPort())) {
-						LOG.info("Killing Kafka Server {}", leaderToShutDown);
-						kafkaServer.shutdown();
-						leaderHasShutDown = true;
-						break;
-					}
-				}
-			}
-		});
-		brokerShutdown.start();
-
-		// add consuming topology:
-		DataStreamSource<String> consuming = env.addSource(new PersistentKafkaSource<String>(topic, new JavaDefaultStringSchema(), standardCC));
-		consuming.setParallelism(1);
-
-		consuming.addSink(new SinkFunction<String>() {
-			private static final long serialVersionUID = 1L;
-
-			int elCnt = 0;
-			int start = 0;
-			int numOfMessagesToBeCorrect = 100;
-			int stopAfterMessages = 150;
-
-			BitSet validator = new BitSet(numOfMessagesToBeCorrect + 1);
-
-			@Override
-			public void invoke(String value) throws Exception {
-				LOG.info("Got message = " + value + " leader has shut down " + leaderHasShutDown + " el cnt = " + elCnt + " to rec" + numOfMessagesToBeCorrect);
-				String[] sp = value.split("-");
-				int v = Integer.parseInt(sp[1]);
-
-				if (start == -1) {
-					start = v;
-				}
-				int offset = v - start;
-				Assert.assertFalse("Received tuple with value " + offset + " twice", validator.get(offset));
-				if (v - start < 0 && LOG.isWarnEnabled()) {
-					LOG.warn("Not in order: {}", value);
-				}
-
-				validator.set(offset);
-				elCnt++;
-				if (elCnt == 20) {
-					LOG.info("Asking leading broker to shut down");
-					// shut down a Kafka broker
-					shutdownKafkaBroker = true;
-				}
-				if (shutdownKafkaBroker) {
-					// we become a bit slower because the shutdown takes some time and we have
-					// only a fixed nubmer of elements to read
-					Thread.sleep(20);
-				}
-				if (leaderHasShutDown) { // it only makes sence to check once the shutdown is completed
-					if (elCnt >= stopAfterMessages) {
-						// check if everything in the bitset is set to true
-						int nc;
-						if ((nc = validator.nextClearBit(0)) < numOfMessagesToBeCorrect) {
-							throw new RuntimeException("The bitset was not set to 1 on all elements to be checked. Next clear:" + nc + " Set: " + validator);
-						}
-						throw new SuccessException();
-					}
-				}
-			}
-		});
-		tryExecute(env, "broker failure test - reader");
-
-	}
-
-	public static void tryExecute(StreamExecutionEnvironment see, String name) throws Exception {
-		try {
-			see.execute(name);
-		} catch (JobExecutionException good) {
-			Throwable t = good.getCause();
-			int limit = 0;
-			while (!(t instanceof SuccessException)) {
-				if(t == null) {
-					LOG.warn("Test failed with exception", good);
-					Assert.fail("Test failed with: " + good.getMessage());
-				}
-
-				t = t.getCause();
-				if (limit++ == 20) {
-					LOG.warn("Test failed with exception", good);
-					Assert.fail("Test failed with: " + good.getMessage());
-				}
-			}
-		}
-	}
-
-	private void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
-		// create topic
-		Properties topicConfig = new Properties();
-		LOG.info("Creating topic {}", topic);
-		AdminUtils.createTopic(zkClient, topic, numberOfPartitions, replicationFactor, topicConfig);
-	}
-
-	private static TestingServer getZookeeper() throws Exception {
-		return new TestingServer(zkPort, tmpZkDir);
-	}
-
-	/**
-	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
-	 */
-	private static KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws UnknownHostException {
-		Properties kafkaProperties = new Properties();
-
-		int kafkaPort = NetUtils.getAvailablePort();
-
-		// properties have to be Strings
-		kafkaProperties.put("advertised.host.name", kafkaHost);
-		kafkaProperties.put("port", Integer.toString(kafkaPort));
-		kafkaProperties.put("broker.id", Integer.toString(brokerId));
-		kafkaProperties.put("log.dir", tmpFolder.toString());
-		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
-		kafkaProperties.put("message.max.bytes", "" + (35 * 1024 * 1024));
-		kafkaProperties.put("replica.fetch.max.bytes", "" + (35 * 1024 * 1024));
-		KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
-
-		KafkaServer server = new KafkaServer(kafkaConfig, new KafkaLocalSystemTime());
-		server.startup();
-		return server;
-	}
-
-	public static class SuccessException extends Exception {
-		private static final long serialVersionUID = 1L;
-	}
-
-
-	// ----------------------- Debugging utilities --------------------
-
-	/**
-	 * Read topic to list, only using Kafka code.
-	 */
-	private static List<MessageAndMetadata<byte[], byte[]>> readTopicToList(String topicName, ConsumerConfig config, final int stopAfter) {
-		ConsumerConnector consumerConnector = Consumer.createJavaConsumerConnector(config);
-		// we request only one stream per consumer instance. Kafka will make sure that each consumer group
-		// will see each message only once.
-		Map<String,Integer> topicCountMap = Collections.singletonMap(topicName, 1);
-		Map<String, List<KafkaStream<byte[], byte[]>>> streams = consumerConnector.createMessageStreams(topicCountMap);
-		if(streams.size() != 1) {
-			throw new RuntimeException("Expected only one message stream but got "+streams.size());
-		}
-		List<KafkaStream<byte[], byte[]>> kafkaStreams = streams.get(topicName);
-		if(kafkaStreams == null) {
-			throw new RuntimeException("Requested stream not available. Available streams: "+streams.toString());
-		}
-		if(kafkaStreams.size() != 1) {
-			throw new RuntimeException("Requested 1 stream from Kafka, bot got "+kafkaStreams.size()+" streams");
-		}
-		LOG.info("Opening Consumer instance for topic '{}' on group '{}'", topicName, config.groupId());
-		ConsumerIterator<byte[], byte[]> iteratorToRead = kafkaStreams.get(0).iterator();
-
-		List<MessageAndMetadata<byte[], byte[]>> result = new ArrayList<MessageAndMetadata<byte[], byte[]>>();
-		int read = 0;
-		while(iteratorToRead.hasNext()) {
-			read++;
-			result.add(iteratorToRead.next());
-			if(read == stopAfter) {
-				LOG.info("Read "+read+" elements");
-				return result;
-			}
-		}
-		return result;
-	}
-
-	private static void printTopic(String topicName, ConsumerConfig config, DeserializationSchema<?> deserializationSchema, int stopAfter){
-		List<MessageAndMetadata<byte[], byte[]>> contents = readTopicToList(topicName, config, stopAfter);
-		LOG.info("Printing contents of topic {} in consumer group {}", 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, ExecutionConfig ec) {
-		// write the sequence to log for debugging purposes
-		Properties stdProps = standardCC.props().props();
-		Properties newProps = new Properties(stdProps);
-		newProps.setProperty("group.id", "topic-printer"+UUID.randomUUID().toString());
-		newProps.setProperty("auto.offset.reset", "smallest");
-		newProps.setProperty("zookeeper.connect", standardCC.zkConnect());
-
-		ConsumerConfig printerConfig = new ConsumerConfig(newProps);
-		TypeInformation<Tuple2<Integer, Integer>> typeInfo = TypeInfoParser.parse("Tuple2<Integer, Integer>");
-		
-		DeserializationSchema<Tuple2<Integer, Integer>> deserializer = 
-				new TypeInformationSerializationSchema<>(typeInfo, ec);
-		printTopic(topicName, printerConfig, deserializer, elements);
-	}
-
-}

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/MockRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/MockRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/MockRuntimeContext.java
new file mode 100644
index 0000000..1f71271
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/MockRuntimeContext.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.streaming.connectors.testutils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.accumulators.DoubleCounter;
+import org.apache.flink.api.common.accumulators.Histogram;
+import org.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.cache.DistributedCache;
+import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.OperatorState;
+import org.apache.flink.api.common.state.StateCheckpointer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+public class MockRuntimeContext implements RuntimeContext {
+
+	private final int numberOfParallelSubtasks;
+	private final int indexOfThisSubtask;
+
+	public MockRuntimeContext(int numberOfParallelSubtasks, int indexOfThisSubtask) {
+		this.numberOfParallelSubtasks = numberOfParallelSubtasks;
+		this.indexOfThisSubtask = indexOfThisSubtask;
+	}
+
+
+	@Override
+	public String getTaskName() {
+		return null;
+	}
+
+	@Override
+	public int getNumberOfParallelSubtasks() {
+		return numberOfParallelSubtasks;
+	}
+
+	@Override
+	public int getIndexOfThisSubtask() {
+		return indexOfThisSubtask;
+	}
+
+	@Override
+	public ExecutionConfig getExecutionConfig() {
+		return null;
+	}
+
+	@Override
+	public ClassLoader getUserCodeClassLoader() {
+		return null;
+	}
+
+	@Override
+	public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator) {}
+
+	@Override
+	public <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name) {
+		return null;
+	}
+
+	@Override
+	public Map<String, Accumulator<?, ?>> getAllAccumulators() {
+		return null;
+	}
+
+	@Override
+	public IntCounter getIntCounter(String name) {
+		return null;
+	}
+
+	@Override
+	public LongCounter getLongCounter(String name) {
+		return null;
+	}
+
+	@Override
+	public DoubleCounter getDoubleCounter(String name) {
+		return null;
+	}
+
+	@Override
+	public Histogram getHistogram(String name) {
+		return null;
+	}
+
+	@Override
+	public <RT> List<RT> getBroadcastVariable(String name) {
+		return null;
+	}
+
+	@Override
+	public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
+		return null;
+	}
+
+	@Override
+	public DistributedCache getDistributedCache() {
+		return null;
+	}
+
+	@Override
+	public <S, C extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState, boolean partitioned, StateCheckpointer<S, C> checkpointer) throws IOException {
+		return null;
+	}
+
+	@Override
+	public <S extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState, boolean partitioned) throws IOException {
+		return null;
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/ThrottledMapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/ThrottledMapper.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/ThrottledMapper.java
new file mode 100644
index 0000000..872d42f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/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.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/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.java
new file mode 100644
index 0000000..1e5f027
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.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.testutils;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.connectors.SerializableKafkaPartitioner;
+
+/**
+ * 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 implements SerializableKafkaPartitioner {
+	
+	private static final long serialVersionUID = 1L;
+
+	private final int expectedPartitions;
+
+	
+	public Tuple2Partitioner(int expectedPartitions) {
+		this.expectedPartitions = expectedPartitions;
+	}
+
+	@Override
+	public int partition(Object key, int numPartitions) {
+		if (numPartitions != expectedPartitions) {
+			throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions");
+		}
+		@SuppressWarnings("unchecked")
+		Tuple2<Integer, Integer> element = (Tuple2<Integer, Integer>) key;
+		
+		return element.f0;
+	}
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
index dc20726..6bdfb48 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
@@ -16,7 +16,7 @@
 # limitations under the License.
 ################################################################################
 
-log4j.rootLogger=OFF, testlogger
+log4j.rootLogger=INFO, testlogger
 
 log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
 log4j.appender.testlogger.target = System.err
@@ -24,4 +24,6 @@ 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
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
index 2bffefd..4c002d1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
@@ -53,7 +53,8 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment {
 	 */
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
-		JobExecutionResult result = ClusterUtil.runOnMiniCluster(getStreamGraph().getJobGraph(), getParallelism(), -1, getConfig().isSysoutLoggingEnabled());
+		JobExecutionResult result = ClusterUtil.runOnMiniCluster(getStreamGraph().getJobGraph(),
+				getParallelism(), -1, getConfig().isSysoutLoggingEnabled(), false, this.conf);
 		transformations.clear();
 		return result;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 906d35d..2352623 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -1196,7 +1196,7 @@ public abstract class StreamExecutionEnvironment {
 	 * @return A local execution environment with the specified parallelism.
 	 */
 	public static LocalStreamEnvironment createLocalEnvironment(int parallelism, Configuration configuration) {
-		currentEnvironment = new LocalStreamEnvironment(configuration);
+		LocalStreamEnvironment currentEnvironment = new LocalStreamEnvironment(configuration);
 		currentEnvironment.setParallelism(parallelism);
 		return (LocalStreamEnvironment) currentEnvironment;
 	}


[38/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Crc32.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Crc32.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Crc32.java
deleted file mode 100644
index 4a23ed6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Crc32.java
+++ /dev/null
@@ -1,396 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.utils;
-
-import java.util.zip.Checksum;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This class was taken from Hadoop org.apache.hadoop.util.PureJavaCrc32
- * 
- * A pure-java implementation of the CRC32 checksum that uses the same polynomial as the built-in native CRC32.
- * 
- * This is to avoid the JNI overhead for certain uses of Checksumming where many small pieces of data are checksummed in
- * succession.
- * 
- * The current version is ~10x to 1.8x as fast as Sun's native java.util.zip.CRC32 in Java 1.6
- * 
- * @see java.util.zip.CRC32
- */
-public class Crc32 implements Checksum {
-
-    /**
-     * Compute the CRC32 of the byte array
-     * 
-     * @param bytes The array to compute the checksum for
-     * @return The CRC32
-     */
-    public static long crc32(byte[] bytes) {
-        return crc32(bytes, 0, bytes.length);
-    }
-
-    /**
-     * Compute the CRC32 of the segment of the byte array given by the specified size and offset
-     * 
-     * @param bytes The bytes to checksum
-     * @param offset the offset at which to begin checksumming
-     * @param size the number of bytes to checksum
-     * @return The CRC32
-     */
-    public static long crc32(byte[] bytes, int offset, int size) {
-        Crc32 crc = new Crc32();
-        crc.update(bytes, offset, size);
-        return crc.getValue();
-    }
-
-    /** the current CRC value, bit-flipped */
-    private int crc;
-
-    /** Create a new PureJavaCrc32 object. */
-    public Crc32() {
-        reset();
-    }
-
-    @Override
-    public long getValue() {
-        return (~crc) & 0xffffffffL;
-    }
-
-    @Override
-    public void reset() {
-        crc = 0xffffffff;
-    }
-
-    @Override
-    public void update(byte[] b, int off, int len) {
-        int localCrc = crc;
-
-        while (len > 7) {
-            final int c0 = (b[off + 0] ^ localCrc) & 0xff;
-            final int c1 = (b[off + 1] ^ (localCrc >>>= 8)) & 0xff;
-            final int c2 = (b[off + 2] ^ (localCrc >>>= 8)) & 0xff;
-            final int c3 = (b[off + 3] ^ (localCrc >>>= 8)) & 0xff;
-            localCrc = (T[T8_7_START + c0] ^ T[T8_6_START + c1]) ^ (T[T8_5_START + c2] ^ T[T8_4_START + c3]);
-
-            final int c4 = b[off + 4] & 0xff;
-            final int c5 = b[off + 5] & 0xff;
-            final int c6 = b[off + 6] & 0xff;
-            final int c7 = b[off + 7] & 0xff;
-
-            localCrc ^= (T[T8_3_START + c4] ^ T[T8_2_START + c5]) ^ (T[T8_1_START + c6] ^ T[T8_0_START + c7]);
-
-            off += 8;
-            len -= 8;
-        }
-
-        /* loop unroll - duff's device style */
-        switch (len) {
-            case 7:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 6:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 5:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 4:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 3:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 2:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            case 1:
-                localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)];
-            default:
-                /* nothing */
-        }
-
-        // Publish crc out to object
-        crc = localCrc;
-    }
-
-    @Override
-    final public void update(int b) {
-        crc = (crc >>> 8) ^ T[T8_0_START + ((crc ^ b) & 0xff)];
-    }
-
-    /**
-     * Update the CRC32 given an integer
-     */
-    final public void updateInt(int input) {
-        update((byte) (input >> 24));
-        update((byte) (input >> 16));
-        update((byte) (input >> 8));
-        update((byte) input /* >> 0 */);
-    }
-
-    /*
-     * CRC-32 lookup tables generated by the polynomial 0xEDB88320. See also TestPureJavaCrc32.Table.
-     */
-    private static final int T8_0_START = 0 * 256;
-    private static final int T8_1_START = 1 * 256;
-    private static final int T8_2_START = 2 * 256;
-    private static final int T8_3_START = 3 * 256;
-    private static final int T8_4_START = 4 * 256;
-    private static final int T8_5_START = 5 * 256;
-    private static final int T8_6_START = 6 * 256;
-    private static final int T8_7_START = 7 * 256;
-
-    private static final int[] T = new int[] {
-        /* T8_0 */
-        0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, 0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, 0x0EDB8832,
-        0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, 0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, 0x1DB71064, 0x6AB020F2,
-        0xF3B97148, 0x84BE41DE, 0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, 0x136C9856, 0x646BA8C0, 0xFD62F97A,
-        0x8A65C9EC, 0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, 0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172,
-        0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, 0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, 0x32D86CE3,
-        0x45DF5C75, 0xDCD60DCF, 0xABD13D59, 0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, 0x21B4F4B5, 0x56B3C423,
-        0xCFBA9599, 0xB8BDA50F, 0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, 0x2F6F7C87, 0x58684C11, 0xC1611DAB,
-        0xB6662D3D, 0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, 0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433,
-        0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, 0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, 0x6B6B51F4,
-        0x1C6C6162, 0x856530D8, 0xF262004E, 0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, 0x65B0D9C6, 0x12B7E950,
-        0x8BBEB8EA, 0xFCB9887C, 0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, 0x4DB26158, 0x3AB551CE, 0xA3BC0074,
-        0xD4BB30E2, 0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, 0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0,
-        0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, 0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, 0x5768B525,
-        0x206F85B3, 0xB966D409, 0xCE61E49F, 0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, 0x59B33D17, 0x2EB40D81,
-        0xB7BD5C3B, 0xC0BA6CAD, 0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, 0xEAD54739, 0x9DD277AF, 0x04DB2615,
-        0x73DC1683, 0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, 0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1,
-        0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, 0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, 0xFED41B76,
-        0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, 0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, 0xD6D6A3E8, 0xA1D1937E,
-        0x38D8C2C4, 0x4FDFF252, 0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, 0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6,
-        0x41047A60, 0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, 0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236,
-        0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, 0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, 0xC2D7FFA7,
-        0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, 0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, 0x9C0906A9, 0xEB0E363F,
-        0x72076785, 0x05005713, 0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, 0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7,
-        0x0BDBDF21, 0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, 0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777,
-        0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, 0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, 0xA00AE278,
-        0xD70DD2EE, 0x4E048354, 0x3903B3C2, 0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, 0xAED16A4A, 0xD9D65ADC,
-        0x40DF0B66, 0x37D83BF0, 0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, 0xBDBDF21C, 0xCABAC28A, 0x53B39330,
-        0x24B4A3A6, 0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, 0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94,
-        0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D,
-        /* T8_1 */
-        0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, 0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, 0xC8D98A08,
-        0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, 0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, 0x4AC21251, 0x53D92310,
-        0x78F470D3, 0x61EF4192, 0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, 0x821B9859, 0x9B00A918, 0xB02DFADB,
-        0xA936CB9A, 0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, 0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761,
-        0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, 0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, 0x39316BAE,
-        0x202A5AEF, 0x0B07092C, 0x121C386D, 0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, 0xBB2AF3F7, 0xA231C2B6,
-        0x891C9175, 0x9007A034, 0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, 0x73F379FF, 0x6AE848BE, 0x41C51B7D,
-        0x58DE2A3C, 0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, 0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2,
-        0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, 0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, 0xBABB5D54,
-        0xA3A06C15, 0x888D3FD6, 0x91960E97, 0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, 0x7262D75C, 0x6B79E61D,
-        0x4054B5DE, 0x594F849F, 0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, 0x65FD6BA7, 0x7CE65AE6, 0x57CB0925,
-        0x4ED03864, 0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, 0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C,
-        0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, 0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, 0x4B53BCF2,
-        0x52488DB3, 0x7965DE70, 0x607EEF31, 0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, 0x838A36FA, 0x9A9107BB,
-        0xB1BC5478, 0xA8A76539, 0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, 0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD,
-        0x74C20E8C, 0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, 0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484,
-        0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, 0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, 0xB9980012,
-        0xA0833153, 0x8BAE6290, 0x92B553D1, 0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, 0xAE07BCE9, 0xB71C8DA8,
-        0x9C31DE6B, 0x852AEF2A, 0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, 0x66DE36E1, 0x7FC507A0, 0x54E85463,
-        0x4DF36522, 0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, 0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B,
-        0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, 0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, 0x4870E1B4,
-        0x516BD0F5, 0x7A468336, 0x635DB277, 0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, 0xAF96124A, 0xB68D230B,
-        0x9DA070C8, 0x84BB4189, 0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, 0x674F9842, 0x7E54A903, 0x5579FAC0,
-        0x4C62CB81, 0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, 0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8,
-        0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, 0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, 0x5E7EF3EC,
-        0x4765C2AD, 0x6C48916E, 0x7553A02F, 0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, 0x96A779E4, 0x8FBC48A5,
-        0xA4911B66, 0xBD8A2A27, 0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, 0x14BCE1BD, 0x0DA7D0FC, 0x268A833F,
-        0x3F91B27E, 0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, 0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876,
-        0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72,
-        /* T8_2 */
-        0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, 0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, 0x0E1351B8,
-        0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, 0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, 0x1C26A370, 0x1DE4C947,
-        0x1FA2771E, 0x1E601D29, 0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, 0x1235F2C8, 0x13F798FF, 0x11B126A6,
-        0x10734C91, 0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, 0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9,
-        0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, 0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, 0x3157BF84,
-        0x3095D5B3, 0x32D36BEA, 0x331101DD, 0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, 0x23624D4C, 0x22A0277B,
-        0x20E69922, 0x2124F315, 0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, 0x2D711CF4, 0x2CB376C3, 0x2EF5C89A,
-        0x2F37A2AD, 0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, 0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45,
-        0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, 0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, 0x6CBC2EB0,
-        0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, 0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, 0x62AF7F08, 0x636D153F,
-        0x612BAB66, 0x60E9C151, 0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, 0x48D7CB20, 0x4915A117, 0x4B531F4E,
-        0x4A917579, 0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, 0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1,
-        0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, 0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, 0x53F8C08C,
-        0x523AAABB, 0x507C14E2, 0x51BE7ED5, 0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, 0x5DEB9134, 0x5C29FB03,
-        0x5E6F455A, 0x5FAD2F6D, 0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, 0xE63CB35C, 0xE7FED96B, 0xE5B86732,
-        0xE47A0D05, 0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, 0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD,
-        0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, 0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, 0xF300E948,
-        0xF2C2837F, 0xF0843D26, 0xF1465711, 0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, 0xD9785D60, 0xD8BA3757,
-        0xDAFC890E, 0xDB3EE339, 0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, 0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6,
-        0xD52DB281, 0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, 0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049,
-        0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, 0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, 0xCC440774,
-        0xCD866D43, 0xCFC0D31A, 0xCE02B92D, 0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, 0x96A63E9C, 0x976454AB,
-        0x9522EAF2, 0x94E080C5, 0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, 0x98B56F24, 0x99770513, 0x9B31BB4A,
-        0x9AF3D17D, 0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, 0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5,
-        0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, 0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, 0xA9E2D0A0,
-        0xA820BA97, 0xAA6604CE, 0xABA46EF9, 0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, 0xA7F18118, 0xA633EB2F,
-        0xA4755576, 0xA5B73F41, 0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, 0xB5C473D0, 0xB40619E7, 0xB640A7BE,
-        0xB782CD89, 0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, 0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31,
-        0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED,
-        /* T8_3 */
-        0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, 0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, 0xC5B428EF,
-        0x7D084F8A, 0x6FBDE064, 0xD7018701, 0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, 0x5019579F, 0xE8A530FA,
-        0xFA109F14, 0x42ACF871, 0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, 0x95AD7F70, 0x2D111815, 0x3FA4B7FB,
-        0x8718D09E, 0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, 0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0,
-        0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, 0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, 0xEAE41086,
-        0x525877E3, 0x40EDD80D, 0xF851BF68, 0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, 0x7F496FF6, 0xC7F50893,
-        0xD540A77D, 0x6DFCC018, 0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, 0xBAFD4719, 0x0241207C, 0x10F48F92,
-        0xA848E8F7, 0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, 0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084,
-        0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, 0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, 0xCB0D0FA2,
-        0x73B168C7, 0x6104C729, 0xD9B8A04C, 0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, 0x0EB9274D, 0xB6054028,
-        0xA4B0EFC6, 0x1C0C88A3, 0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, 0x3B26F703, 0x839A9066, 0x912F3F88,
-        0x299358ED, 0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, 0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002,
-        0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, 0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, 0xE45D37CB,
-        0x5CE150AE, 0x4E54FF40, 0xF6E89825, 0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, 0x21E91F24, 0x99557841,
-        0x8BE0D7AF, 0x335CB0CA, 0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, 0x623B216C, 0xDA874609, 0xC832E9E7,
-        0x708E8E82, 0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, 0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D,
-        0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, 0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, 0x78F4C94B,
-        0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, 0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, 0x4D6B1905, 0xF5D77E60,
-        0xE762D18E, 0x5FDEB6EB, 0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, 0x88DF31EA, 0x3063568F, 0x22D6F961,
-        0x9A6A9E04, 0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, 0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174,
-        0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, 0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, 0x57A4F122,
-        0xEF189647, 0xFDAD39A9, 0x45115ECC, 0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, 0xF92F7951, 0x41931E34,
-        0x5326B1DA, 0xEB9AD6BF, 0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, 0x3C9B51BE, 0x842736DB, 0x96929935,
-        0x2E2EFE50, 0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, 0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120,
-        0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, 0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, 0xD67F4138,
-        0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, 0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, 0x13CB69D7, 0xAB770EB2,
-        0xB9C2A15C, 0x017EC639, 0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, 0x866616A7, 0x3EDA71C2, 0x2C6FDE2C,
-        0x94D3B949, 0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, 0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6,
-        0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1,
-        /* T8_4 */
-        0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, 0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, 0x30704BC1,
-        0x0D106271, 0x4AB018A1, 0x77D03111, 0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, 0x60E09782, 0x5D80BE32,
-        0x1A20C4E2, 0x2740ED52, 0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, 0x5090DC43, 0x6DF0F5F3, 0x2A508F23,
-        0x1730A693, 0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, 0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4,
-        0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, 0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, 0x0431C205,
-        0x3951EBB5, 0x7EF19165, 0x4391B8D5, 0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, 0x54A11E46, 0x69C137F6,
-        0x2E614D26, 0x13016496, 0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, 0x64D15587, 0x59B17C37, 0x1E1106E7,
-        0x23712F57, 0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, 0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459,
-        0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, 0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, 0x3813CFCB,
-        0x0573E67B, 0x42D39CAB, 0x7FB3B51B, 0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, 0x0863840A, 0x3503ADBA,
-        0x72A3D76A, 0x4FC3FEDA, 0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, 0x9932774D, 0xA4525EFD, 0xE3F2242D,
-        0xDE920D9D, 0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, 0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C,
-        0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, 0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, 0x0C52460F,
-        0x31326FBF, 0x7692156F, 0x4BF23CDF, 0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, 0x3C220DCE, 0x0142247E,
-        0x46E25EAE, 0x7B82771E, 0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, 0x44661652, 0x79063FE2, 0x3EA64532,
-        0x03C66C82, 0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, 0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743,
-        0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, 0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, 0xE1766CD1,
-        0xDC164561, 0x9BB63FB1, 0xA6D61601, 0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, 0x70279F96, 0x4D47B626,
-        0x0AE7CCF6, 0x3787E546, 0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, 0x4057D457, 0x7D37FDE7, 0x3A978737,
-        0x07F7AE87, 0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, 0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4,
-        0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, 0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, 0xD537E515,
-        0xE857CCA5, 0xAFF7B675, 0x92979FC5, 0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, 0x1C954E1B, 0x21F567AB,
-        0x66551D7B, 0x5B3534CB, 0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, 0x2CE505DA, 0x11852C6A, 0x562556BA,
-        0x6B457F0A, 0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, 0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349,
-        0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, 0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, 0x28D4C7DF,
-        0x15B4EE6F, 0x521494BF, 0x6F74BD0F, 0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, 0x18A48C1E, 0x25C4A5AE,
-        0x6264DF7E, 0x5F04F6CE, 0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, 0x4834505D, 0x755479ED, 0x32F4033D,
-        0x0F942A8D, 0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, 0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C,
-        0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C,
-        /* T8_5 */
-        0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, 0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, 0xEC53826D,
-        0x270F51C8, 0xA19B2366, 0x6AC7F0C3, 0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, 0x03D6029B, 0xC88AD13E,
-        0x4E1EA390, 0x85427035, 0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, 0xEF8580F6, 0x24D95353, 0xA24D21FD,
-        0x6911F258, 0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, 0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798,
-        0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, 0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, 0x706EC54D,
-        0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, 0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, 0x9FEB45BB, 0x54B7961E,
-        0xD223E4B0, 0x197F3715, 0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, 0x73B8C7D6, 0xB8E41473, 0x3E7066DD,
-        0xF52CB578, 0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, 0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4,
-        0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, 0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, 0x0C8E08F7,
-        0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, 0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, 0xE0DD8A9A, 0x2B81593F,
-        0xAD152B91, 0x6649F834, 0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, 0x08F40F5A, 0xC3A8DCFF, 0x453CAE51,
-        0x8E607DF4, 0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, 0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99,
-        0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, 0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, 0x90B34FD7,
-        0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, 0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, 0x7CE0CDBA, 0xB7BC1E1F,
-        0x31286CB1, 0xFA74BF14, 0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, 0x852156CE, 0x4E7D856B, 0xC8E9F7C5,
-        0x03B52460, 0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, 0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D,
-        0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, 0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, 0xF135942E,
-        0x3A69478B, 0xBCFD3525, 0x77A1E680, 0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, 0x191C11EE, 0xD240C24B,
-        0x54D4B0E5, 0x9F886340, 0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, 0xF54F9383, 0x3E134026, 0xB8873288,
-        0x73DBE12D, 0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, 0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB,
-        0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, 0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, 0x6D08D30E,
-        0xA65400AB, 0x20C07205, 0xEB9CA1A0, 0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, 0x8A795CA2, 0x41258F07,
-        0xC7B1FDA9, 0x0CED2E0C, 0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, 0x662ADECF, 0xAD760D6A, 0x2BE27FC4,
-        0xE0BEAC61, 0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, 0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97,
-        0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, 0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, 0x16441B82,
-        0xDD18C827, 0x5B8CBA89, 0x90D0692C, 0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, 0xFA1799EF, 0x314B4A4A,
-        0xB7DF38E4, 0x7C83EB41, 0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, 0x15921919, 0xDECECABC, 0x585AB812,
-        0x93066BB7, 0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, 0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA,
-        0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC,
-        /* T8_6 */
-        0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, 0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, 0x33EF4E67,
-        0x959845D3, 0xA4705F4E, 0x020754FA, 0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, 0x67DE9CCE, 0xC1A9977A,
-        0xF0418DE7, 0x56368653, 0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, 0x5431D2A9, 0xF246D91D, 0xC3AEC380,
-        0x65D9C834, 0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, 0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301,
-        0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, 0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, 0x081D53E8,
-        0xAE6A585C, 0x9F8242C1, 0x39F54975, 0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, 0x5C2C8141, 0xFA5B8AF5,
-        0xCBB39068, 0x6DC49BDC, 0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, 0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F,
-        0x5E2BD5BB, 0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, 0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7,
-        0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, 0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, 0x23D5E9B7,
-        0x85A2E203, 0xB44AF89E, 0x123DF32A, 0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, 0x103AA7D0, 0xB64DAC64,
-        0x87A5B6F9, 0x21D2BD4D, 0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, 0x8BB64CE5, 0x2DC14751, 0x1C295DCC,
-        0xBA5E5678, 0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, 0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F,
-        0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, 0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, 0x1827F438,
-        0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, 0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, 0x2BC8BA5F, 0x8DBFB1EB,
-        0xBC57AB76, 0x1A20A0C2, 0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, 0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8,
-        0x4DB1D47C, 0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, 0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B,
-        0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, 0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, 0xDC27385B,
-        0x7A5033EF, 0x4BB82972, 0xEDCF22C6, 0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, 0x47ABD36E, 0xE1DCD8DA,
-        0xD034C247, 0x7643C9F3, 0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, 0x74449D09, 0xD23396BD, 0xE3DB8C20,
-        0x45AC8794, 0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, 0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D,
-        0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, 0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, 0xE7D525D4,
-        0x41A22E60, 0x704A34FD, 0xD63D3F49, 0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, 0x3852BB98, 0x9E25B02C,
-        0xAFCDAAB1, 0x09BAA105, 0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, 0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6,
-        0x3A55EF62, 0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, 0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB,
-        0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, 0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, 0x03A0A617,
-        0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, 0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, 0x304FE870, 0x9638E3C4,
-        0xA7D0F959, 0x01A7F2ED, 0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, 0x647E3AD9, 0xC209316D, 0xF3E12BF0,
-        0x55962044, 0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, 0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23,
-        0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30,
-        /* T8_7 */
-        0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, 0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, 0xD3E51BB5,
-        0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, 0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, 0x7CBB312B, 0xB01131B5,
-        0x3E9E3656, 0xF23436C8, 0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, 0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3,
-        0x21D12D7D, 0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, 0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5,
-        0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, 0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, 0xAED97719,
-        0x62737787, 0xECFC7064, 0x205670FA, 0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, 0x01875D87, 0xCD2D5D19,
-        0x43A25AFA, 0x8F085A64, 0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, 0xD2624632, 0x1EC846AC, 0x9047414F,
-        0x5CED41D1, 0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, 0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4,
-        0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, 0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, 0x5526F3C6,
-        0x998CF358, 0x1703F4BB, 0xDBA9F425, 0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, 0x86C3E873, 0x4A69E8ED,
-        0xC4E6EF0E, 0x084CEF90, 0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, 0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6,
-        0x5E64A758, 0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, 0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED,
-        0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, 0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, 0x281A9F6A,
-        0xE4B09FF4, 0x6A3F9817, 0xA6959889, 0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, 0xFBFF84DF, 0x37558441,
-        0xB9DA83A2, 0x7570833C, 0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, 0xD7718B20, 0x1BDB8BBE, 0x95548C5D,
-        0x59FE8CC3, 0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, 0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776,
-        0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, 0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, 0xFC65AF44,
-        0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, 0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, 0xAA4DE78C, 0x66E7E712,
-        0xE868E0F1, 0x24C2E06F, 0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, 0x79A8FC39, 0xB502FCA7, 0x3B8DFB44,
-        0xF727FBDA, 0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, 0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144,
-        0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, 0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, 0x8159C3E8,
-        0x4DF3C376, 0xC37CC495, 0x0FD6C40B, 0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, 0xFEEC49CD, 0x32464953,
-        0xBCC94EB0, 0x70634E2E, 0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, 0x2D095278, 0xE1A352E6, 0x6F2C5505,
-        0xA386559B, 0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, 0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05,
-        0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, 0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, 0x83D02561,
-        0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, 0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, 0x50353ED4, 0x9C9F3E4A,
-        0x121039A9, 0xDEBA3937, 0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, 0xFF6B144A, 0x33C114D4, 0xBD4E1337,
-        0x71E413A9, 0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, 0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C,
-        0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6 };
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/KafkaThread.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/KafkaThread.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/KafkaThread.java
deleted file mode 100644
index 310c39b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/KafkaThread.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.kafka_backport.common.utils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A wrapper for Thread that sets things up nicely
- */
-public class KafkaThread extends Thread {
-
-    private final Logger log = LoggerFactory.getLogger(getClass());
-
-    public KafkaThread(final String name, Runnable runnable, boolean daemon) {
-        super(runnable, name);
-        setDaemon(daemon);
-        setUncaughtExceptionHandler(new UncaughtExceptionHandler() {
-            public void uncaughtException(Thread t, Throwable e) {
-                log.error("Uncaught exception in " + name + ": ", e);
-            }
-        });
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/SystemTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/SystemTime.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/SystemTime.java
deleted file mode 100644
index 428258c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/SystemTime.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.kafka_backport.common.utils;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A time implementation that uses the system clock and sleep call
- */
-public class SystemTime implements Time {
-
-    @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) {
-            // just wake up early
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Time.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Time.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Time.java
deleted file mode 100644
index 88bb1c0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Time.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.kafka_backport.common.utils;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * An interface abstracting the clock to use in unit testing classes that make use of clock time
- */
-public interface Time {
-
-    /**
-     * The current time in milliseconds
-     */
-    public long milliseconds();
-
-    /**
-     * The current time in nanoseconds
-     */
-    public long nanoseconds();
-
-    /**
-     * Sleep for the given number of milliseconds
-     */
-    public void sleep(long ms);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Utils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Utils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Utils.java
deleted file mode 100755
index e2a5463..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/Utils.java
+++ /dev/null
@@ -1,506 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.utils;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.io.UnsupportedEncodingException;
-import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Properties;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class Utils {
-
-    // This matches URIs of formats: host:port and protocol:\\host:port
-    // IPv6 is supported with [ip] pattern
-    private static final Pattern HOST_PORT_PATTERN = Pattern.compile(".*?\\[?([0-9a-z\\-.:]*)\\]?:([0-9]+)");
-
-    public static final String NL = System.getProperty("line.separator");
-
-    private static final Logger log = LoggerFactory.getLogger(Utils.class);
-
-    /**
-     * Turn the given UTF8 byte array into a string
-     * 
-     * @param bytes The byte array
-     * @return The string
-     */
-    public static String utf8(byte[] bytes) {
-        try {
-            return new String(bytes, "UTF8");
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException("This shouldn't happen.", e);
-        }
-    }
-
-    /**
-     * Turn a string into a utf8 byte[]
-     * 
-     * @param string The string
-     * @return The byte[]
-     */
-    public static byte[] utf8(String string) {
-        try {
-            return string.getBytes("UTF8");
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException("This shouldn't happen.", e);
-        }
-    }
-
-    /**
-     * Read an unsigned integer from the current position in the buffer, incrementing the position by 4 bytes
-     * 
-     * @param buffer The buffer to read from
-     * @return The integer read, as a long to avoid signedness
-     */
-    public static long readUnsignedInt(ByteBuffer buffer) {
-        return buffer.getInt() & 0xffffffffL;
-    }
-
-    /**
-     * Read an unsigned integer from the given position without modifying the buffers position
-     * 
-     * @param buffer the buffer to read from
-     * @param index the index from which to read the integer
-     * @return The integer read, as a long to avoid signedness
-     */
-    public static long readUnsignedInt(ByteBuffer buffer, int index) {
-        return buffer.getInt(index) & 0xffffffffL;
-    }
-
-    /**
-     * Read an unsigned integer stored in little-endian format from the {@link InputStream}.
-     * 
-     * @param in The stream to read from
-     * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS)
-     */
-    public static int readUnsignedIntLE(InputStream in) throws IOException {
-        return (in.read() << 8 * 0) 
-             | (in.read() << 8 * 1)
-             | (in.read() << 8 * 2)
-             | (in.read() << 8 * 3);
-    }
-
-    /**
-     * Read an unsigned integer stored in little-endian format from a byte array
-     * at a given offset.
-     * 
-     * @param buffer The byte array to read from
-     * @param offset The position in buffer to read from
-     * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS)
-     */
-    public static int readUnsignedIntLE(byte[] buffer, int offset) {
-        return (buffer[offset++] << 8 * 0)
-             | (buffer[offset++] << 8 * 1)
-             | (buffer[offset++] << 8 * 2)
-             | (buffer[offset]   << 8 * 3);
-    }
-
-    /**
-     * Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
-     * 
-     * @param buffer The buffer to write to
-     * @param value The value to write
-     */
-    public static void writetUnsignedInt(ByteBuffer buffer, long value) {
-        buffer.putInt((int) (value & 0xffffffffL));
-    }
-
-    /**
-     * Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
-     * 
-     * @param buffer The buffer to write to
-     * @param index The position in the buffer at which to begin writing
-     * @param value The value to write
-     */
-    public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) {
-        buffer.putInt(index, (int) (value & 0xffffffffL));
-    }
-
-    /**
-     * Write an unsigned integer in little-endian format to the {@link OutputStream}.
-     * 
-     * @param out The stream to write to
-     * @param value The value to write
-     */
-    public static void writeUnsignedIntLE(OutputStream out, int value) throws IOException {
-        out.write(value >>> 8 * 0);
-        out.write(value >>> 8 * 1);
-        out.write(value >>> 8 * 2);
-        out.write(value >>> 8 * 3);
-    }
-
-    /**
-     * Write an unsigned integer in little-endian format to a byte array
-     * at a given offset.
-     * 
-     * @param buffer The byte array to write to
-     * @param offset The position in buffer to write to
-     * @param value The value to write
-     */
-    public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) {
-        buffer[offset++] = (byte) (value >>> 8 * 0);
-        buffer[offset++] = (byte) (value >>> 8 * 1);
-        buffer[offset++] = (byte) (value >>> 8 * 2);
-        buffer[offset]   = (byte) (value >>> 8 * 3);
-    }
-
-
-    /**
-     * Get the absolute value of the given number. If the number is Int.MinValue return 0. This is different from
-     * java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!).
-     */
-    public static int abs(int n) {
-        return (n == Integer.MIN_VALUE) ? 0 : Math.abs(n);
-    }
-
-    /**
-     * Get the minimum of some long values.
-     * @param first Used to ensure at least one value
-     * @param rest The rest of longs to compare
-     * @return The minimum of all passed argument.
-     */
-    public static long min(long first, long ... rest) {
-        long min = first;
-        for (int i = 0; i < rest.length; i++) {
-            if (rest[i] < min)
-                min = rest[i];
-        }
-        return min;
-    }
-
-    /**
-     * Get the length for UTF8-encoding a string without encoding it first
-     * 
-     * @param s The string to calculate the length for
-     * @return The length when serialized
-     */
-    public static int utf8Length(CharSequence s) {
-        int count = 0;
-        for (int i = 0, len = s.length(); i < len; i++) {
-            char ch = s.charAt(i);
-            if (ch <= 0x7F) {
-                count++;
-            } else if (ch <= 0x7FF) {
-                count += 2;
-            } else if (Character.isHighSurrogate(ch)) {
-                count += 4;
-                ++i;
-            } else {
-                count += 3;
-            }
-        }
-        return count;
-    }
-
-    /**
-     * Read the given byte buffer into a byte array
-     */
-    public static byte[] toArray(ByteBuffer buffer) {
-        return toArray(buffer, 0, buffer.limit());
-    }
-
-    /**
-     * Read a byte array from the given offset and size in the buffer
-     */
-    public static byte[] toArray(ByteBuffer buffer, int offset, int size) {
-        byte[] dest = new byte[size];
-        if (buffer.hasArray()) {
-            System.arraycopy(buffer.array(), buffer.arrayOffset() + offset, dest, 0, size);
-        } else {
-            int pos = buffer.position();
-            buffer.get(dest);
-            buffer.position(pos);
-        }
-        return dest;
-    }
-
-    /**
-     * Check that the parameter t is not null
-     * 
-     * @param t The object to check
-     * @return t if it isn't null
-     * @throws NullPointerException if t is null.
-     */
-    public static <T> T notNull(T t) {
-        if (t == null)
-            throw new NullPointerException();
-        else
-            return t;
-    }
-
-    /**
-     * Sleep for a bit
-     * @param ms The duration of the sleep
-     */
-    public static void sleep(long ms) {
-        try {
-            Thread.sleep(ms);
-        } catch (InterruptedException e) {
-            // this is okay, we just wake up early
-        }
-    }
-
-    /**
-     * Instantiate the class
-     */
-    public static Object newInstance(Class<?> c) {
-        try {
-            return c.newInstance();
-        } catch (IllegalAccessException e) {
-            throw new KafkaException("Could not instantiate class " + c.getName(), e);
-        } catch (InstantiationException e) {
-            throw new KafkaException("Could not instantiate class " + c.getName() + " Does it have a public no-argument constructor?", e);
-        }
-    }
-
-    /**
-     * Generates 32 bit murmur2 hash from byte array
-     * @param data byte array to hash
-     * @return 32 bit hash of the given array
-     */
-    public static int murmur2(final byte[] data) {
-        int length = data.length;
-        int seed = 0x9747b28c;
-        // 'm' and 'r' are mixing constants generated offline.
-        // They're not really 'magic', they just happen to work well.
-        final int m = 0x5bd1e995;
-        final int r = 24;
-
-        // Initialize the hash to a random value
-        int h = seed ^ length;
-        int length4 = length / 4;
-
-        for (int i = 0; i < length4; i++) {
-            final int i4 = i * 4;
-            int k = (data[i4 + 0] & 0xff) + ((data[i4 + 1] & 0xff) << 8) + ((data[i4 + 2] & 0xff) << 16) + ((data[i4 + 3] & 0xff) << 24);
-            k *= m;
-            k ^= k >>> r;
-            k *= m;
-            h *= m;
-            h ^= k;
-        }
-
-        // Handle the last few bytes of the input array
-        switch (length % 4) {
-            case 3:
-                h ^= (data[(length & ~3) + 2] & 0xff) << 16;
-            case 2:
-                h ^= (data[(length & ~3) + 1] & 0xff) << 8;
-            case 1:
-                h ^= data[length & ~3] & 0xff;
-                h *= m;
-        }
-
-        h ^= h >>> 13;
-        h *= m;
-        h ^= h >>> 15;
-
-        return h;
-    }
-
-    /**
-     * Extracts the hostname from a "host:port" address string.
-     * @param address address string to parse
-     * @return hostname or null if the given address is incorrect
-     */
-    public static String getHost(String address) {
-        Matcher matcher = HOST_PORT_PATTERN.matcher(address);
-        return matcher.matches() ? matcher.group(1) : null;
-    }
-
-    /**
-     * Extracts the port number from a "host:port" address string.
-     * @param address address string to parse
-     * @return port number or null if the given address is incorrect
-     */
-    public static Integer getPort(String address) {
-        Matcher matcher = HOST_PORT_PATTERN.matcher(address);
-        return matcher.matches() ? Integer.parseInt(matcher.group(2)) : null;
-    }
-
-    /**
-     * Formats hostname and port number as a "host:port" address string,
-     * surrounding IPv6 addresses with braces '[', ']'
-     * @param host hostname
-     * @param port port number
-     * @return address string
-     */
-    public static String formatAddress(String host, Integer port) {
-        return host.contains(":")
-                ? "[" + host + "]:" + port // IPv6
-                : host + ":" + port;
-    }
-
-    /**
-     * Create a string representation of an array joined by the given separator
-     * @param strs The array of items
-     * @param seperator The separator
-     * @return The string representation.
-     */
-    public static <T> String join(T[] strs, String seperator) {
-        return join(Arrays.asList(strs), seperator);
-    }
-    
-    /**
-     * Create a string representation of a list joined by the given separator
-     * @param list The list of items
-     * @param seperator The separator
-     * @return The string representation.
-     */
-    public static <T> String join(Collection<T> list, String seperator) {
-        StringBuilder sb = new StringBuilder();
-        Iterator<T> iter = list.iterator();
-        while (iter.hasNext()) {
-            sb.append(iter.next());
-            if (iter.hasNext())
-                sb.append(seperator);  
-        }
-        return sb.toString();
-    }
-
-    /**
-     * Read a properties file from the given path
-     * @param filename The path of the file to read
-     */
-    public static Properties loadProps(String filename) throws IOException, FileNotFoundException {
-        Properties props = new Properties();
-        InputStream propStream = null;
-        try {
-            propStream = new FileInputStream(filename);
-            props.load(propStream);
-        } finally {
-            if (propStream != null)
-                propStream.close();
-        }
-        return props;
-    }
-
-    /**
-     * Get the stack trace from an exception as a string
-     */
-    public static String stackTrace(Throwable e) {
-        StringWriter sw = new StringWriter();
-        PrintWriter pw = new PrintWriter(sw);
-        e.printStackTrace(pw);
-        return sw.toString();
-    }
-
-    /**
-     * Create a new thread
-     * @param name The name of the thread
-     * @param runnable The work for the thread to do
-     * @param daemon Should the thread block JVM shutdown?
-     * @return The unstarted thread
-     */
-    public static Thread newThread(String name, Runnable runnable, Boolean daemon) {
-        Thread thread = new Thread(runnable, name);
-        thread.setDaemon(daemon);
-        thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
-            public void uncaughtException(Thread t, Throwable e) {
-                log.error("Uncaught exception in thread '" + t.getName() + "':", e);
-            }
-        });
-        return thread;
-    }
-
-    /**
-     * Create a daemon thread
-     * @param name The name of the thread
-     * @param runnable The runnable to execute in the background
-     * @return The unstarted thread
-     */
-    public static Thread daemonThread(String name, Runnable runnable) {
-        return newThread(name, runnable, true);
-    }
-
-    /**
-     * Print an error message and shutdown the JVM
-     * @param message The error message
-     */
-    public static void croak(String message) {
-        System.err.println(message);
-        System.exit(1);
-    }
-
-    /**
-     * Read a buffer into a Byte array for the given offset and length
-     */
-    public static byte[] readBytes(ByteBuffer buffer, int offset, int length) {
-        byte[] dest = new byte[length];
-        if (buffer.hasArray()) {
-            System.arraycopy(buffer.array(), buffer.arrayOffset() + offset, dest, 0, length);
-        } else {
-            buffer.mark();
-            buffer.position(offset);
-            buffer.get(dest, 0, length);
-            buffer.reset();
-        }
-        return dest;
-    }
-
-    /**
-     * Read the given byte buffer into a Byte array
-     */
-    public static byte[] readBytes(ByteBuffer buffer) {
-        return Utils.readBytes(buffer, 0, buffer.limit());
-    }
-
-    /**
-     * Attempt to read a file as a string
-     * @throws IOException 
-     */
-    public static String readFileAsString(String path, Charset charset) throws IOException {
-        if (charset == null) charset = Charset.defaultCharset();
-        FileInputStream stream = new FileInputStream(new File(path));
-        String result = new String();
-        try {
-            FileChannel fc = stream.getChannel();
-            MappedByteBuffer bb = fc.map(FileChannel.MapMode.READ_ONLY, 0, fc.size());
-            result = charset.decode(bb).toString();
-        } finally {
-            stream.close();
-        }
-        return result;
-    }
-
-    public static String readFileAsString(String path) throws IOException {
-        return Utils.readFileAsString(path, Charset.defaultCharset());
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
deleted file mode 100644
index 6a574c8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/package-info.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- *  This package (and its sub-packages) contain only classes that arecopied from
- *  the Apache Kafka project.
- *
- *  The package contains a "backport" of the new consumer API, in order to
- *  give Flink access to its functionality until the API is properly released.
- *
- *  This is a temporary workaround!
- */
-package org.apache.flink.kafka_backport;

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

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


[42/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Protocol.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Protocol.java
deleted file mode 100644
index f7c8981..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/Protocol.java
+++ /dev/null
@@ -1,474 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.protocol;
-
-import org.apache.flink.kafka_backport.common.protocol.types.ArrayOf;
-import org.apache.flink.kafka_backport.common.protocol.types.Field;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Type;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class Protocol {
-
-    public static final Schema REQUEST_HEADER = new Schema(new Field("api_key", Type.INT16, "The id of the request type."),
-                                                           new Field("api_version", Type.INT16, "The version of the API."),
-                                                           new Field("correlation_id",
-                                                                     Type.INT32,
-                                                                     "A user-supplied integer value that will be passed back with the response"),
-                                                           new Field("client_id",
-                                                                     Type.STRING,
-                                                                     "A user specified identifier for the client making the request."));
-
-    public static final Schema RESPONSE_HEADER = new Schema(new Field("correlation_id",
-                                                                      Type.INT32,
-                                                                      "The user-supplied value passed in with the request"));
-
-    /* Metadata api */
-
-    public static final Schema METADATA_REQUEST_V0 = new Schema(new Field("topics",
-                                                                          new ArrayOf(Type.STRING),
-                                                                          "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics."));
-
-    public static final Schema BROKER = new Schema(new Field("node_id", Type.INT32, "The broker id."),
-                                                   new Field("host", Type.STRING, "The hostname of the broker."),
-                                                   new Field("port",
-                                                             Type.INT32,
-                                                             "The port on which the broker accepts requests."));
-
-    public static final Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code",
-                                                                            Type.INT16,
-                                                                            "The error code for the partition, if any."),
-                                                                  new Field("partition_id",
-                                                                            Type.INT32,
-                                                                            "The id of the partition."),
-                                                                  new Field("leader",
-                                                                            Type.INT32,
-                                                                            "The id of the broker acting as leader for this partition."),
-                                                                  new Field("replicas",
-                                                                            new ArrayOf(Type.INT32),
-                                                                            "The set of all nodes that host this partition."),
-                                                                  new Field("isr",
-                                                                            new ArrayOf(Type.INT32),
-                                                                            "The set of nodes that are in sync with the leader for this partition."));
-
-    public static final Schema TOPIC_METADATA_V0 = new Schema(new Field("topic_error_code",
-                                                                        Type.INT16,
-                                                                        "The error code for the given topic."),
-                                                              new Field("topic", Type.STRING, "The name of the topic"),
-                                                              new Field("partition_metadata",
-                                                                        new ArrayOf(PARTITION_METADATA_V0),
-                                                                        "Metadata for each partition of the topic."));
-
-    public static final Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers",
-                                                                           new ArrayOf(BROKER),
-                                                                           "Host and port information for all brokers."),
-                                                                 new Field("topic_metadata",
-                                                                           new ArrayOf(TOPIC_METADATA_V0)));
-
-    public static final Schema[] METADATA_REQUEST = new Schema[] {METADATA_REQUEST_V0};
-    public static final Schema[] METADATA_RESPONSE = new Schema[] {METADATA_RESPONSE_V0};
-
-    /* Produce api */
-
-    public static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", Type.STRING),
-                                                                  new Field("data", new ArrayOf(new Schema(new Field("partition", Type.INT32),
-                                                                                                     new Field("record_set", Type.BYTES)))));
-
-    public static final Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks",
-                                                                   Type.INT16,
-                                                                   "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."),
-                                                               new Field("timeout", Type.INT32, "The time to await a response in ms."),
-                                                               new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0)));
-
-    public static final Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                    new ArrayOf(new Schema(new Field("topic", Type.STRING),
-                                                                                           new Field("partition_responses",
-                                                                                                     new ArrayOf(new Schema(new Field("partition",
-                                                                                                                                      Type.INT32),
-                                                                                                                            new Field("error_code",
-                                                                                                                                      Type.INT16),
-                                                                                                                            new Field("base_offset",
-                                                                                                                                      Type.INT64))))))));
-
-    public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0};
-    public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0};
-
-    /* Offset commit api */
-    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                         Type.INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         Type.INT64,
-                                                                                         "Message offset to be committed."),
-                                                                               new Field("metadata",
-                                                                                         Type.STRING,
-                                                                                         "Any associated metadata the client wants to keep."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition",
-                                                                                         Type.INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         Type.INT64,
-                                                                                         "Message offset to be committed."),
-                                                                               new Field("timestamp",
-                                                                                         Type.INT64,
-                                                                                         "Timestamp of the commit"),
-                                                                               new Field("metadata",
-                                                                                         Type.STRING,
-                                                                                         "Any associated metadata the client wants to keep."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V2 = new Schema(new Field("partition",
-                                                                                         Type.INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         Type.INT64,
-                                                                                         "Message offset to be committed."),
-                                                                               new Field("metadata",
-                                                                                         Type.STRING,
-                                                                                         "Any associated metadata the client wants to keep."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
-                                                                                     Type.STRING,
-                                                                                     "Topic to commit."),
-                                                                           new Field("partitions",
-                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0),
-                                                                                     "Partitions to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(new Field("topic",
-                                                                                     Type.STRING,
-                                                                                     "Topic to commit."),
-                                                                           new Field("partitions",
-                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1),
-                                                                                     "Partitions to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V2 = new Schema(new Field("topic",
-                                                                                     Type.STRING,
-                                                                                     "Topic to commit."),
-                                                                           new Field("partitions",
-                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V2),
-                                                                                     "Partitions to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                               Type.STRING,
-                                                                               "The consumer group id."),
-                                                                     new Field("topics",
-                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
-                                                                               "Topics to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id",
-                                                                               Type.STRING,
-                                                                               "The consumer group id."),
-                                                                     new Field("group_generation_id",
-                                                                               Type.INT32,
-                                                                               "The generation of the consumer group."),
-                                                                     new Field("consumer_id",
-                                                                               Type.STRING,
-                                                                               "The consumer id assigned by the group coordinator."),
-                                                                     new Field("topics",
-                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1),
-                                                                               "Topics to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id",
-                                                                               Type.STRING,
-                                                                               "The consumer group id."),
-                                                                     new Field("group_generation_id",
-                                                                               Type.INT32,
-                                                                               "The generation of the consumer group."),
-                                                                     new Field("consumer_id",
-                                                                               Type.STRING,
-                                                                               "The consumer id assigned by the group coordinator."),
-                                                                     new Field("retention_time",
-                                                                               Type.INT64,
-                                                                               "Time period in ms to retain the offset."),
-                                                                     new Field("topics",
-                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2),
-                                                                               "Topics to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                          Type.INT32,
-                                                                                          "Topic partition id."),
-                                                                                new Field("error_code",
-                                                                                          Type.INT16));
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
-                                                                            new Field("partition_responses",
-                                                                                      new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0)));
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                                new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] OFFSET_COMMIT_REQUEST = new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2};
-
-    /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */
-    public static final Schema OFFSET_COMMIT_RESPONSE_V1 = OFFSET_COMMIT_RESPONSE_V0;
-    public static final Schema OFFSET_COMMIT_RESPONSE_V2 = OFFSET_COMMIT_RESPONSE_V0;
-
-    public static final Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V1, OFFSET_COMMIT_RESPONSE_V2};
-
-    /* Offset fetch api */
-
-    /*
-     * Wire formats of version 0 and 1 are the same, but with different functionality.
-     * Version 0 will read the offsets from ZK;
-     * Version 1 will read the offsets from Kafka.
-     */
-    public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                        Type.INT32,
-                                                                                        "Topic partition id."));
-
-    public static final Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
-                                                                                    Type.STRING,
-                                                                                    "Topic to fetch offset."),
-                                                                          new Field("partitions",
-                                                                                    new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0),
-                                                                                    "Partitions to fetch offsets."));
-
-    public static final Schema OFFSET_FETCH_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                              Type.STRING,
-                                                                              "The consumer group id."),
-                                                                    new Field("topics",
-                                                                              new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0),
-                                                                              "Topics to fetch offsets."));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                         Type.INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         Type.INT64,
-                                                                                         "Last committed message offset."),
-                                                                               new Field("metadata",
-                                                                                         Type.STRING,
-                                                                                         "Any associated metadata the client wants to keep."),
-                                                                               new Field("error_code", Type.INT16));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
-                                                                           new Field("partition_responses",
-                                                                                     new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0)));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                               new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)));
-
-    public static final Schema OFFSET_FETCH_REQUEST_V1 = OFFSET_FETCH_REQUEST_V0;
-    public static final Schema OFFSET_FETCH_RESPONSE_V1 = OFFSET_FETCH_RESPONSE_V0;
-
-    public static final Schema[] OFFSET_FETCH_REQUEST = new Schema[] {OFFSET_FETCH_REQUEST_V0, OFFSET_FETCH_REQUEST_V1};
-    public static final Schema[] OFFSET_FETCH_RESPONSE = new Schema[] {OFFSET_FETCH_RESPONSE_V0, OFFSET_FETCH_RESPONSE_V1};
-
-    /* List offset api */
-    public static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                       Type.INT32,
-                                                                                       "Topic partition id."),
-                                                                             new Field("timestamp", Type.INT64, "Timestamp."),
-                                                                             new Field("max_num_offsets",
-                                                                                       Type.INT32,
-                                                                                       "Maximum offsets to return."));
-
-    public static final Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
-                                                                                   Type.STRING,
-                                                                                   "Topic to list offset."),
-                                                                         new Field("partitions",
-                                                                                   new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0),
-                                                                                   "Partitions to list offset."));
-
-    public static final Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id",
-                                                                             Type.INT32,
-                                                                             "Broker id of the follower. For normal consumers, use -1."),
-                                                                   new Field("topics",
-                                                                             new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0),
-                                                                             "Topics to list offsets."));
-
-    public static final Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                        Type.INT32,
-                                                                                        "Topic partition id."),
-                                                                              new Field("error_code", Type.INT16),
-                                                                              new Field("offsets",
-                                                                                        new ArrayOf(Type.INT64),
-                                                                                        "A list of offsets."));
-
-    public static final Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
-                                                                          new Field("partition_responses",
-                                                                                    new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0)));
-
-    public static final Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                              new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] LIST_OFFSET_REQUEST = new Schema[] {LIST_OFFSET_REQUEST_V0};
-    public static final Schema[] LIST_OFFSET_RESPONSE = new Schema[] {LIST_OFFSET_RESPONSE_V0};
-
-    /* Fetch api */
-    public static final Schema FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                 Type.INT32,
-                                                                                 "Topic partition id."),
-                                                                       new Field("fetch_offset",
-                                                                                 Type.INT64,
-                                                                                 "Message offset."),
-                                                                       new Field("max_bytes",
-                                                                                 Type.INT32,
-                                                                                 "Maximum bytes to fetch."));
-
-    public static final Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", Type.STRING, "Topic to fetch."),
-                                                                   new Field("partitions",
-                                                                             new ArrayOf(FETCH_REQUEST_PARTITION_V0),
-                                                                             "Partitions to fetch."));
-
-    public static final Schema FETCH_REQUEST_V0 = new Schema(new Field("replica_id",
-                                                                       Type.INT32,
-                                                                       "Broker id of the follower. For normal consumers, use -1."),
-                                                             new Field("max_wait_time",
-                                                                       Type.INT32,
-                                                                       "Maximum time in ms to wait for the response."),
-                                                             new Field("min_bytes",
-                                                                       Type.INT32,
-                                                                       "Minimum bytes to accumulate in the response."),
-                                                             new Field("topics",
-                                                                       new ArrayOf(FETCH_REQUEST_TOPIC_V0),
-                                                                       "Topics to fetch."));
-
-    public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                  Type.INT32,
-                                                                                  "Topic partition id."),
-                                                                        new Field("error_code", Type.INT16),
-                                                                        new Field("high_watermark",
-                                                                                  Type.INT64,
-                                                                                  "Last committed offset."),
-                                                                        new Field("record_set", Type.BYTES));
-
-    public static final Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
-                                                                    new Field("partition_responses",
-                                                                              new ArrayOf(FETCH_RESPONSE_PARTITION_V0)));
-
-    public static final Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                        new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0};
-    public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0};
-
-    /* Consumer metadata api */
-    public static final Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                                   Type.STRING,
-                                                                                   "The consumer group id."));
-
-    public static final Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", Type.INT16),
-                                                                          new Field("coordinator",
-                                                                                    BROKER,
-                                                                                    "Host and port information for the coordinator for a consumer group."));
-
-    public static final Schema[] CONSUMER_METADATA_REQUEST = new Schema[] {CONSUMER_METADATA_REQUEST_V0};
-    public static final Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] {CONSUMER_METADATA_RESPONSE_V0};
-
-    /* Join group api */
-    public static final Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                            Type.STRING,
-                                                                            "The consumer group id."),
-                                                                  new Field("session_timeout",
-                                                                            Type.INT32,
-                                                                            "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."),
-                                                                  new Field("topics",
-                                                                            new ArrayOf(Type.STRING),
-                                                                            "An array of topics to subscribe to."),
-                                                                  new Field("consumer_id",
-                                                                            Type.STRING,
-                                                                            "The assigned consumer id or an empty string for a new consumer."),
-                                                                  new Field("partition_assignment_strategy",
-                                                                            Type.STRING,
-                                                                            "The strategy for the coordinator to assign partitions."));
-
-    public static final Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", Type.STRING),
-                                                                         new Field("partitions", new ArrayOf(Type.INT32)));
-    public static final Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", Type.INT16),
-                                                                   new Field("group_generation_id",
-                                                                             Type.INT32,
-                                                                             "The generation of the consumer group."),
-                                                                   new Field("consumer_id",
-                                                                             Type.STRING,
-                                                                             "The consumer id assigned by the group coordinator."),
-                                                                   new Field("assigned_partitions",
-                                                                             new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] JOIN_GROUP_REQUEST = new Schema[] {JOIN_GROUP_REQUEST_V0};
-    public static final Schema[] JOIN_GROUP_RESPONSE = new Schema[] {JOIN_GROUP_RESPONSE_V0};
-
-    /* Heartbeat api */
-    public static final Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id", Type.STRING, "The consumer group id."),
-                                                                 new Field("group_generation_id",
-                                                                           Type.INT32,
-                                                                           "The generation of the consumer group."),
-                                                                 new Field("consumer_id",
-                                                                           Type.STRING,
-                                                                           "The consumer id assigned by the group coordinator."));
-
-    public static final Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", Type.INT16));
-
-    public static final Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0};
-    public static final Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0};
-
-    /* an array of all requests and responses with all schema versions */
-    public static final Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][];
-    public static final Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][];
-
-    /* the latest version of each api */
-    public static final short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1];
-
-    static {
-        REQUESTS[ApiKeys.PRODUCE.id] = PRODUCE_REQUEST;
-        REQUESTS[ApiKeys.FETCH.id] = FETCH_REQUEST;
-        REQUESTS[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_REQUEST;
-        REQUESTS[ApiKeys.METADATA.id] = METADATA_REQUEST;
-        REQUESTS[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
-        REQUESTS[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
-        REQUESTS[ApiKeys.UPDATE_METADATA_KEY.id] = new Schema[] {};
-        REQUESTS[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = new Schema[] {};
-        REQUESTS[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_REQUEST;
-        REQUESTS[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_REQUEST;
-        REQUESTS[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_REQUEST;
-        REQUESTS[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_REQUEST;
-        REQUESTS[ApiKeys.HEARTBEAT.id] = HEARTBEAT_REQUEST;
-
-        RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
-        RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE;
-        RESPONSES[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_RESPONSE;
-        RESPONSES[ApiKeys.METADATA.id] = METADATA_RESPONSE;
-        RESPONSES[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
-        RESPONSES[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
-        RESPONSES[ApiKeys.UPDATE_METADATA_KEY.id] = new Schema[] {};
-        RESPONSES[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = new Schema[] {};
-        RESPONSES[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_RESPONSE;
-        RESPONSES[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_RESPONSE;
-        RESPONSES[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_RESPONSE;
-        RESPONSES[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_RESPONSE;
-        RESPONSES[ApiKeys.HEARTBEAT.id] = HEARTBEAT_RESPONSE;
-
-        /* set the maximum version of each api */
-        for (ApiKeys api : ApiKeys.values())
-            CURR_VERSION[api.id] = (short) (REQUESTS[api.id].length - 1);
-
-        /* sanity check that we have the same number of request and response versions for each api */
-        for (ApiKeys api : ApiKeys.values())
-            if (REQUESTS[api.id].length != RESPONSES[api.id].length)
-                throw new IllegalStateException(REQUESTS[api.id].length + " request versions for api " + api.name
-                        + " but " + RESPONSES[api.id].length + " response versions.");
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/SecurityProtocol.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/SecurityProtocol.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/SecurityProtocol.java
deleted file mode 100644
index ab5a607..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/SecurityProtocol.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.kafka_backport.common.protocol;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public enum SecurityProtocol {
-    /** Un-authenticated, non-encrypted channel */
-    PLAINTEXT(0, "PLAINTEXT"),
-    /** Currently identical to PLAINTEXT and used for testing only. We may implement extra instrumentation when testing channel code. */
-    TRACE(Short.MAX_VALUE, "TRACE");
-
-    private static final Map<Short, SecurityProtocol> CODE_TO_SECURITY_PROTOCOL = new HashMap<Short, SecurityProtocol>();
-    private static final List<String> NAMES = new ArrayList<String>();
-
-    static {
-        for (SecurityProtocol proto: SecurityProtocol.values()) {
-            CODE_TO_SECURITY_PROTOCOL.put(proto.id, proto);
-            NAMES.add(proto.name);
-        }
-    }
-
-    /** The permanent and immutable id of a security protocol -- this can't change, and must match kafka.cluster.SecurityProtocol  */
-    public final short id;
-
-    /** Name of the security protocol. This may be used by client configuration. */
-    public final String name;
-
-    private SecurityProtocol(int id, String name) {
-        this.id = (short) id;
-        this.name = name;
-    }
-
-    public static String getName(int id) {
-        return CODE_TO_SECURITY_PROTOCOL.get((short) id).name;
-    }
-
-    public static List<String> getNames() {
-        return NAMES;
-    }
-
-    public static SecurityProtocol forId(Short id) {
-        return CODE_TO_SECURITY_PROTOCOL.get(id);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/ArrayOf.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/ArrayOf.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/ArrayOf.java
deleted file mode 100644
index d2468d8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/ArrayOf.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.protocol.types;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Represents a type for an array of a particular type
- */
-public class ArrayOf extends Type {
-
-    private final Type type;
-
-    public ArrayOf(Type type) {
-        this.type = type;
-    }
-
-    @Override
-    public void write(ByteBuffer buffer, Object o) {
-        Object[] objs = (Object[]) o;
-        int size = objs.length;
-        buffer.putInt(size);
-        for (int i = 0; i < size; i++)
-            type.write(buffer, objs[i]);
-    }
-
-    @Override
-    public Object read(ByteBuffer buffer) {
-        int size = buffer.getInt();
-        Object[] objs = new Object[size];
-        for (int i = 0; i < size; i++)
-            objs[i] = type.read(buffer);
-        return objs;
-    }
-
-    @Override
-    public int sizeOf(Object o) {
-        Object[] objs = (Object[]) o;
-        int size = 4;
-        for (int i = 0; i < objs.length; i++)
-            size += type.sizeOf(objs[i]);
-        return size;
-    }
-
-    public Type type() {
-        return type;
-    }
-
-    @Override
-    public String toString() {
-        return "ARRAY(" + type + ")";
-    }
-
-    @Override
-    public Object[] validate(Object item) {
-        try {
-            Object[] array = (Object[]) item;
-            for (int i = 0; i < array.length; i++)
-                type.validate(array[i]);
-            return array;
-        } catch (ClassCastException e) {
-            throw new SchemaException("Not an Object[].");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Field.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Field.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Field.java
deleted file mode 100644
index b7d7720..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Field.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.kafka_backport.common.protocol.types;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A field in a schema
- */
-public class Field {
-
-    public static final Object NO_DEFAULT = new Object();
-
-    final int index;
-    public final String name;
-    public final Type type;
-    public final Object defaultValue;
-    public final String doc;
-    final Schema schema;
-
-    /**
-     * Create the field.
-     *
-     * @throws SchemaException If the default value is not primitive and the validation fails
-     */
-    public Field(int index, String name, Type type, String doc, Object defaultValue, Schema schema) {
-        this.index = index;
-        this.name = name;
-        this.type = type;
-        this.doc = doc;
-        this.defaultValue = defaultValue;
-        this.schema = schema;
-        if (defaultValue != NO_DEFAULT)
-            type.validate(defaultValue);
-    }
-
-    public Field(int index, String name, Type type, String doc, Object defaultValue) {
-        this(index, name, type, doc, defaultValue, null);
-    }
-
-    public Field(String name, Type type, String doc, Object defaultValue) {
-        this(-1, name, type, doc, defaultValue);
-    }
-
-    public Field(String name, Type type, String doc) {
-        this(name, type, doc, NO_DEFAULT);
-    }
-
-    public Field(String name, Type type) {
-        this(name, type, "");
-    }
-
-    public Type type() {
-        return type;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Schema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Schema.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Schema.java
deleted file mode 100644
index 7adac52..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Schema.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.kafka_backport.common.protocol.types;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The schema for a compound record definition
- */
-public class Schema extends Type {
-
-    private final Field[] fields;
-    private final Map<String, Field> fieldsByName;
-
-    /**
-     * Construct the schema with a given list of its field values
-     *
-     * @throws SchemaException If the given list have duplicate fields
-     */
-    public Schema(Field... fs) {
-        this.fields = new Field[fs.length];
-        this.fieldsByName = new HashMap<String, Field>();
-        for (int i = 0; i < this.fields.length; i++) {
-            Field field = fs[i];
-            if (fieldsByName.containsKey(field.name))
-                throw new SchemaException("Schema contains a duplicate field: " + field.name);
-            this.fields[i] = new Field(i, field.name, field.type, field.doc, field.defaultValue, this);
-            this.fieldsByName.put(fs[i].name, this.fields[i]);
-        }
-    }
-
-    /**
-     * Write a struct to the buffer
-     */
-    public void write(ByteBuffer buffer, Object o) {
-        Struct r = (Struct) o;
-        for (int i = 0; i < fields.length; i++) {
-            Field f = fields[i];
-            try {
-                Object value = f.type().validate(r.get(f));
-                f.type.write(buffer, value);
-            } catch (Exception e) {
-                throw new SchemaException("Error writing field '" + f.name +
-                                          "': " +
-                                          (e.getMessage() == null ? e.getClass().getName() : e.getMessage()));
-            }
-        }
-    }
-
-    /**
-     * Read a struct from the buffer
-     */
-    public Object read(ByteBuffer buffer) {
-        Object[] objects = new Object[fields.length];
-        for (int i = 0; i < fields.length; i++) {
-            try {
-                objects[i] = fields[i].type.read(buffer);
-            } catch (Exception e) {
-                throw new SchemaException("Error reading field '" + fields[i].name +
-                                          "': " +
-                                          (e.getMessage() == null ? e.getClass().getName() : e.getMessage()));
-            }
-        }
-        return new Struct(this, objects);
-    }
-
-    /**
-     * The size of the given record
-     */
-    public int sizeOf(Object o) {
-        int size = 0;
-        Struct r = (Struct) o;
-        for (int i = 0; i < fields.length; i++)
-            size += fields[i].type.sizeOf(r.get(fields[i]));
-        return size;
-    }
-
-    /**
-     * The number of fields in this schema
-     */
-    public int numFields() {
-        return this.fields.length;
-    }
-
-    /**
-     * Get a field by its slot in the record array
-     * 
-     * @param slot The slot at which this field sits
-     * @return The field
-     */
-    public Field get(int slot) {
-        return this.fields[slot];
-    }
-
-    /**
-     * Get a field by its name
-     * 
-     * @param name The name of the field
-     * @return The field
-     */
-    public Field get(String name) {
-        return this.fieldsByName.get(name);
-    }
-
-    /**
-     * Get all the fields in this schema
-     */
-    public Field[] fields() {
-        return this.fields;
-    }
-
-    /**
-     * Display a string representation of the schema
-     */
-    public String toString() {
-        StringBuilder b = new StringBuilder();
-        b.append('{');
-        for (int i = 0; i < this.fields.length; i++) {
-            b.append(this.fields[i].name);
-            b.append(':');
-            b.append(this.fields[i].type());
-            if (i < this.fields.length - 1)
-                b.append(',');
-        }
-        b.append("}");
-        return b.toString();
-    }
-
-    @Override
-    public Struct validate(Object item) {
-        try {
-            Struct struct = (Struct) item;
-            for (int i = 0; i < this.fields.length; i++) {
-                Field field = this.fields[i];
-                try {
-                    field.type.validate(struct.get(field));
-                } catch (SchemaException e) {
-                    throw new SchemaException("Invalid value for field '" + field.name + "': " + e.getMessage());
-                }
-            }
-            return struct;
-        } catch (ClassCastException e) {
-            throw new SchemaException("Not a Struct.");
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/SchemaException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/SchemaException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/SchemaException.java
deleted file mode 100644
index 86c141e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/SchemaException.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.kafka_backport.common.protocol.types;
-
-import org.apache.flink.kafka_backport.common.KafkaException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- *  Thrown if the protocol schema validation fails while parsing request or response.
- */
-public class SchemaException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public SchemaException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Struct.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Struct.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Struct.java
deleted file mode 100644
index 482fe9d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Struct.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.kafka_backport.common.protocol.types;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A record that can be serialized and deserialized according to a pre-defined schema
- */
-public class Struct {
-    private final Schema schema;
-    private final Object[] values;
-
-    Struct(Schema schema, Object[] values) {
-        this.schema = schema;
-        this.values = values;
-    }
-
-    public Struct(Schema schema) {
-        this.schema = schema;
-        this.values = new Object[this.schema.numFields()];
-    }
-
-    /**
-     * The schema for this struct.
-     */
-    public Schema schema() {
-        return this.schema;
-    }
-
-    /**
-     * Return the value of the given pre-validated field, or if the value is missing return the default value.
-     * 
-     * @param field The field for which to get the default value
-     * @throws SchemaException if the field has no value and has no default.
-     */
-    private Object getFieldOrDefault(Field field) {
-        Object value = this.values[field.index];
-        if (value != null)
-            return value;
-        else if (field.defaultValue != Field.NO_DEFAULT)
-            return field.defaultValue;
-        else
-            throw new SchemaException("Missing value for field '" + field.name + "' which has no default value.");
-    }
-
-    /**
-     * Get the value for the field directly by the field index with no lookup needed (faster!)
-     * 
-     * @param field The field to look up
-     * @return The value for that field.
-     * @throws SchemaException if the field has no value and has no default.
-     */
-    public Object get(Field field) {
-        validateField(field);
-        return getFieldOrDefault(field);
-    }
-
-    /**
-     * Get the record value for the field with the given name by doing a hash table lookup (slower!)
-     * 
-     * @param name The name of the field
-     * @return The value in the record
-     * @throws SchemaException If no such field exists
-     */
-    public Object get(String name) {
-        Field field = schema.get(name);
-        if (field == null)
-            throw new SchemaException("No such field: " + name);
-        return getFieldOrDefault(field);
-    }
-
-    /**
-     * Check if the struct contains a field.
-     * @param name
-     * @return Whether a field exists.
-     */
-    public boolean hasField(String name) {
-        return schema.get(name) != null;
-    }
-
-    public Struct getStruct(Field field) {
-        return (Struct) get(field);
-    }
-
-    public Struct getStruct(String name) {
-        return (Struct) get(name);
-    }
-
-    public Short getShort(Field field) {
-        return (Short) get(field);
-    }
-
-    public Short getShort(String name) {
-        return (Short) get(name);
-    }
-
-    public Integer getInt(Field field) {
-        return (Integer) get(field);
-    }
-
-    public Integer getInt(String name) {
-        return (Integer) get(name);
-    }
-
-    public Long getLong(Field field) {
-        return (Long) get(field);
-    }
-
-    public Long getLong(String name) {
-        return (Long) get(name);
-    }
-
-    public Object[] getArray(Field field) {
-        return (Object[]) get(field);
-    }
-
-    public Object[] getArray(String name) {
-        return (Object[]) get(name);
-    }
-
-    public String getString(Field field) {
-        return (String) get(field);
-    }
-
-    public String getString(String name) {
-        return (String) get(name);
-    }
-
-    public ByteBuffer getBytes(Field field) {
-        return (ByteBuffer) get(field);
-    }
-
-    public ByteBuffer getBytes(String name) {
-        return (ByteBuffer) get(name);
-    }
-
-    /**
-     * Set the given field to the specified value
-     * 
-     * @param field The field
-     * @param value The value
-     * @throws SchemaException If the validation of the field failed
-     */
-    public Struct set(Field field, Object value) {
-        validateField(field);
-        this.values[field.index] = value;
-        return this;
-    }
-
-    /**
-     * Set the field specified by the given name to the value
-     * 
-     * @param name The name of the field
-     * @param value The value to set
-     * @throws SchemaException If the field is not known
-     */
-    public Struct set(String name, Object value) {
-        Field field = this.schema.get(name);
-        if (field == null)
-            throw new SchemaException("Unknown field: " + name);
-        this.values[field.index] = value;
-        return this;
-    }
-
-    /**
-     * Create a struct for the schema of a container type (struct or array). Note that for array type, this method
-     * assumes that the type is an array of schema and creates a struct of that schema. Arrays of other types can't be
-     * instantiated with this method.
-     * 
-     * @param field The field to create an instance of
-     * @return The struct
-     * @throws SchemaException If the given field is not a container type
-     */
-    public Struct instance(Field field) {
-        validateField(field);
-        if (field.type() instanceof Schema) {
-            return new Struct((Schema) field.type());
-        } else if (field.type() instanceof ArrayOf) {
-            ArrayOf array = (ArrayOf) field.type();
-            return new Struct((Schema) array.type());
-        } else {
-            throw new SchemaException("Field '" + field.name + "' is not a container type, it is of type " + field.type());
-        }
-    }
-
-    /**
-     * Create a struct instance for the given field which must be a container type (struct or array)
-     * 
-     * @param field The name of the field to create (field must be a schema type)
-     * @return The struct
-     * @throws SchemaException If the given field is not a container type
-     */
-    public Struct instance(String field) {
-        return instance(schema.get(field));
-    }
-
-    /**
-     * Empty all the values from this record
-     */
-    public void clear() {
-        Arrays.fill(this.values, null);
-    }
-
-    /**
-     * Get the serialized size of this object
-     */
-    public int sizeOf() {
-        return this.schema.sizeOf(this);
-    }
-
-    /**
-     * Write this struct to a buffer
-     */
-    public void writeTo(ByteBuffer buffer) {
-        this.schema.write(buffer, this);
-    }
-
-    /**
-     * Ensure the user doesn't try to access fields from the wrong schema
-     *
-     * @throws SchemaException If validation fails
-     */
-    private void validateField(Field field) {
-        if (this.schema != field.schema)
-            throw new SchemaException("Attempt to access field '" + field.name + "' from a different schema instance.");
-        if (field.index > values.length)
-            throw new SchemaException("Invalid field index: " + field.index);
-    }
-
-    /**
-     * Validate the contents of this struct against its schema
-     *
-     * @throws SchemaException If validation fails
-     */
-    public void validate() {
-        this.schema.validate(this);
-    }
-
-    /**
-     * Create a byte buffer containing the serialized form of the values in this struct. This method can choose to break
-     * the struct into multiple ByteBuffers if need be.
-     */
-    public ByteBuffer[] toBytes() {
-        ByteBuffer buffer = ByteBuffer.allocate(sizeOf());
-        writeTo(buffer);
-        return new ByteBuffer[] {buffer};
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder b = new StringBuilder();
-        b.append('{');
-        for (int i = 0; i < this.values.length; i++) {
-            Field f = this.schema.get(i);
-            b.append(f.name);
-            b.append('=');
-            if (f.type() instanceof ArrayOf) {
-                Object[] arrayValue = (Object[]) this.values[i];
-                b.append('[');
-                for (int j = 0; j < arrayValue.length; j++) {
-                    b.append(arrayValue[j]);
-                    if (j < arrayValue.length - 1)
-                        b.append(',');
-                }
-                b.append(']');
-            } else
-                b.append(this.values[i]);
-            if (i < this.values.length - 1)
-                b.append(',');
-        }
-        b.append('}');
-        return b.toString();
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        for (int i = 0; i < this.values.length; i++) {
-            Field f = this.schema.get(i);
-            if (f.type() instanceof ArrayOf) {
-                Object[] arrayObject = (Object[]) this.get(f);
-                for (Object arrayItem: arrayObject)
-                    result = prime * result + arrayItem.hashCode();
-            } else {
-                result = prime * result + this.get(f).hashCode();
-            }
-        }
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        Struct other = (Struct) obj;
-        if (schema != other.schema)
-            return false;
-        for (int i = 0; i < this.values.length; i++) {
-            Field f = this.schema.get(i);
-            Boolean result;
-            if (f.type() instanceof ArrayOf) {
-                result = Arrays.equals((Object[]) this.get(f), (Object[]) other.get(f));
-            } else {
-                result = this.get(f).equals(other.get(f));
-            }
-            if (!result)
-                return false;
-        }
-        return true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Type.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Type.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Type.java
deleted file mode 100644
index 26bdd2f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/protocol/types/Type.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.protocol.types;
-
-import org.apache.flink.kafka_backport.common.utils.Utils;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A serializable type
- */
-public abstract class Type {
-
-    /**
-     * Write the typed object to the buffer
-     *
-     * @throws SchemaException If the object is not valid for its type
-     */
-    public abstract void write(ByteBuffer buffer, Object o);
-
-    /**
-     * Read the typed object from the buffer
-     *
-     * @throws SchemaException If the object is not valid for its type
-     */
-    public abstract Object read(ByteBuffer buffer);
-
-    /**
-     * Validate the object. If succeeded return its typed object.
-     *
-     * @throws SchemaException If validation failed
-     */
-    public abstract Object validate(Object o);
-
-    /**
-     * Return the size of the object in bytes
-     */
-    public abstract int sizeOf(Object o);
-
-    public static final Type INT8 = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            buffer.put((Byte) o);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            return buffer.get();
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 1;
-        }
-
-        @Override
-        public String toString() {
-            return "INT8";
-        }
-
-        @Override
-        public Byte validate(Object item) {
-            if (item instanceof Byte)
-                return (Byte) item;
-            else
-                throw new SchemaException(item + " is not a Byte.");
-        }
-    };
-
-    public static final Type INT16 = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            buffer.putShort((Short) o);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            return buffer.getShort();
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 2;
-        }
-
-        @Override
-        public String toString() {
-            return "INT16";
-        }
-
-        @Override
-        public Short validate(Object item) {
-            if (item instanceof Short)
-                return (Short) item;
-            else
-                throw new SchemaException(item + " is not a Short.");
-        }
-    };
-
-    public static final Type INT32 = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            buffer.putInt((Integer) o);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            return buffer.getInt();
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 4;
-        }
-
-        @Override
-        public String toString() {
-            return "INT32";
-        }
-
-        @Override
-        public Integer validate(Object item) {
-            if (item instanceof Integer)
-                return (Integer) item;
-            else
-                throw new SchemaException(item + " is not an Integer.");
-        }
-    };
-
-    public static final Type INT64 = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            buffer.putLong((Long) o);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            return buffer.getLong();
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 8;
-        }
-
-        @Override
-        public String toString() {
-            return "INT64";
-        }
-
-        @Override
-        public Long validate(Object item) {
-            if (item instanceof Long)
-                return (Long) item;
-            else
-                throw new SchemaException(item + " is not a Long.");
-        }
-    };
-
-    public static final Type STRING = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            byte[] bytes = Utils.utf8((String) o);
-            if (bytes.length > Short.MAX_VALUE)
-                throw new SchemaException("String is longer than the maximum string length.");
-            buffer.putShort((short) bytes.length);
-            buffer.put(bytes);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            int length = buffer.getShort();
-            byte[] bytes = new byte[length];
-            buffer.get(bytes);
-            return Utils.utf8(bytes);
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 2 + Utils.utf8Length((String) o);
-        }
-
-        @Override
-        public String toString() {
-            return "STRING";
-        }
-
-        @Override
-        public String validate(Object item) {
-            if (item instanceof String)
-                return (String) item;
-            else
-                throw new SchemaException(item + " is not a String.");
-        }
-    };
-
-    public static final Type BYTES = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            ByteBuffer arg = (ByteBuffer) o;
-            int pos = arg.position();
-            buffer.putInt(arg.remaining());
-            buffer.put(arg);
-            arg.position(pos);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            int size = buffer.getInt();
-            ByteBuffer val = buffer.slice();
-            val.limit(size);
-            buffer.position(buffer.position() + size);
-            return val;
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            ByteBuffer buffer = (ByteBuffer) o;
-            return 4 + buffer.remaining();
-        }
-
-        @Override
-        public String toString() {
-            return "BYTES";
-        }
-
-        @Override
-        public ByteBuffer validate(Object item) {
-            if (item instanceof ByteBuffer)
-                return (ByteBuffer) item;
-            else
-                throw new SchemaException(item + " is not a java.nio.ByteBuffer.");
-        }
-    };
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferInputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferInputStream.java
deleted file mode 100644
index 99a20a3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferInputStream.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.record;
-
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A byte buffer backed input outputStream
- */
-public class ByteBufferInputStream extends InputStream {
-
-    private ByteBuffer buffer;
-
-    public ByteBufferInputStream(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    public int read() {
-        if (!buffer.hasRemaining()) {
-            return -1;
-        }
-        return buffer.get() & 0xFF;
-    }
-
-    public int read(byte[] bytes, int off, int len) {
-        if (!buffer.hasRemaining()) {
-            return -1;
-        }
-
-        len = Math.min(len, buffer.remaining());
-        buffer.get(bytes, off, len);
-        return len;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferOutputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferOutputStream.java
deleted file mode 100644
index a334755..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/ByteBufferOutputStream.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.kafka_backport.common.record;
-
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A byte buffer backed output outputStream
- */
-public class ByteBufferOutputStream extends OutputStream {
-
-    private static final float REALLOCATION_FACTOR = 1.1f;
-
-    private ByteBuffer buffer;
-
-    public ByteBufferOutputStream(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    public void write(int b) {
-        if (buffer.remaining() < 1)
-            expandBuffer(buffer.capacity() + 1);
-        buffer.put((byte) b);
-    }
-
-    public void write(byte[] bytes, int off, int len) {
-        if (buffer.remaining() < len)
-            expandBuffer(buffer.capacity() + len);
-        buffer.put(bytes, off, len);
-    }
-
-    public ByteBuffer buffer() {
-        return buffer;
-    }
-
-    private void expandBuffer(int size) {
-        int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
-        ByteBuffer temp = ByteBuffer.allocate(expandSize);
-        temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
-        buffer = temp;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/CompressionType.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/CompressionType.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/CompressionType.java
deleted file mode 100644
index 9961766..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/CompressionType.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.kafka_backport.common.record;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The compression type to use
- */
-public enum CompressionType {
-    NONE(0, "none", 1.0f), GZIP(1, "gzip", 0.5f), SNAPPY(2, "snappy", 0.5f), LZ4(3, "lz4", 0.5f);
-
-    public final int id;
-    public final String name;
-    public final float rate;
-
-    private CompressionType(int id, String name, float rate) {
-        this.id = id;
-        this.name = name;
-        this.rate = rate;
-    }
-
-    public static CompressionType forId(int id) {
-        switch (id) {
-            case 0:
-                return NONE;
-            case 1:
-                return GZIP;
-            case 2:
-                return SNAPPY;
-            case 3:
-                return LZ4;
-            default:
-                throw new IllegalArgumentException("Unknown compression type id: " + id);
-        }
-    }
-
-    public static CompressionType forName(String name) {
-        if (NONE.name.equals(name))
-            return NONE;
-        else if (GZIP.name.equals(name))
-            return GZIP;
-        else if (SNAPPY.name.equals(name))
-            return SNAPPY;
-        else if (LZ4.name.equals(name))
-            return LZ4;
-        else
-            throw new IllegalArgumentException("Unknown compression name: " + name);
-    }
-
-}


[30/51] [abbrv] flink git commit: [hotfix] Determing native byte order via ByteOrder.nativeOrder()

Posted by se...@apache.org.
[hotfix] Determing native byte order via ByteOrder.nativeOrder()

This closes #1065


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

Branch: refs/heads/master
Commit: 08500f262234445113c6074d3ba29ffb68f1aa33
Parents: 844e0b2
Author: Suminda Dharmasena <si...@gmail.com>
Authored: Thu Aug 27 11:41:57 2015 +0530
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Aug 27 12:40:38 2015 +0200

----------------------------------------------------------------------
 .../org/apache/flink/core/memory/MemoryUtils.java    | 15 +--------------
 1 file changed, 1 insertion(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/08500f26/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
index 0ae12c7..c1f626f 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
@@ -57,20 +57,7 @@ public class MemoryUtils {
 	
 	@SuppressWarnings("restriction")
 	private static ByteOrder getByteOrder() {
-		final byte[] bytes = new byte[8];
-		final long value = 0x12345678900abdefL;
-		UNSAFE.putLong(bytes, (long) UNSAFE.arrayBaseOffset(byte[].class), value);
-		
-		final int lower = bytes[0] & 0xff;
-		final int higher = bytes[7] & 0xff;
-		
-		if (lower == 0x12 && higher == 0xef) {
-			return ByteOrder.BIG_ENDIAN;
-		} else if (lower == 0xef && higher == 0x12) {
-			return ByteOrder.LITTLE_ENDIAN;
-		} else {
-			throw new RuntimeException("Unrecognized byte order.");
-		}
+		return ByteOrder.nativeOrder();
 	}
 	
 	


[39/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchRequest.java
deleted file mode 100644
index e97ebc2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchRequest.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * This wrapper supports both v0 and v1 of OffsetFetchRequest.
- */
-public class OffsetFetchRequest extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id);
-    private static final String GROUP_ID_KEY_NAME = "group_id";
-    private static final String TOPICS_KEY_NAME = "topics";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-
-    private final String groupId;
-    private final List<TopicPartition> partitions;
-
-    public OffsetFetchRequest(String groupId, List<TopicPartition> partitions) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        Map<String, List<Integer>> topicsData = CollectionUtils.groupDataByTopic(partitions);
-
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, List<Integer>> entries: topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Integer partiitonId : entries.getValue()) {
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partiitonId);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
-        this.groupId = groupId;
-        this.partitions = partitions;
-    }
-
-    public OffsetFetchRequest(Struct struct) {
-        super(struct);
-        partitions = new ArrayList<TopicPartition>();
-        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                partitions.add(new TopicPartition(topic, partition));
-            }
-        }
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = new HashMap<TopicPartition, OffsetFetchResponse.PartitionData>();
-
-        for (TopicPartition partition: partitions) {
-            responseData.put(partition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET,
-                    OffsetFetchResponse.NO_METADATA,
-                    Errors.forException(e).code()));
-        }
-
-        switch (versionId) {
-            // OffsetFetchResponseV0 == OffsetFetchResponseV1
-            case 0:
-            case 1:
-                return new OffsetFetchResponse(responseData);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public List<TopicPartition> partitions() {
-        return partitions;
-    }
-
-    public static OffsetFetchRequest parse(ByteBuffer buffer, int versionId) {
-        return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer));
-    }
-
-    public static OffsetFetchRequest parse(ByteBuffer buffer) {
-        return new OffsetFetchRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchResponse.java
deleted file mode 100644
index a1be70f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetFetchResponse.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class OffsetFetchResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level fields
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    // partition level fields
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
-    private static final String METADATA_KEY_NAME = "metadata";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    public static final long INVALID_OFFSET = -1L;
-    public static final String NO_METADATA = "";
-
-    /**
-     * Possible error code:
-     *
-     *  UNKNOWN_TOPIC_OR_PARTITION (3)  <- only for request v0
-     *  OFFSET_LOAD_IN_PROGRESS (14)
-     *  NOT_COORDINATOR_FOR_CONSUMER (16)
-     *  ILLEGAL_GENERATION (22)
-     *  UNKNOWN_CONSUMER_ID (25)
-     */
-
-    private final Map<TopicPartition, PartitionData> responseData;
-
-    public static final class PartitionData {
-        public final long offset;
-        public final String metadata;
-        public final short errorCode;
-
-        public PartitionData(long offset, String metadata, short errorCode) {
-            this.offset = offset;
-            this.metadata = metadata;
-            this.errorCode = errorCode;
-        }
-
-        public boolean hasError() {
-            return this.errorCode != Errors.NONE.code();
-        }
-    }
-
-    public OffsetFetchResponse(Map<TopicPartition, PartitionData> responseData) {
-        super(new Struct(CURRENT_SCHEMA));
-
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
-
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> entries : topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionData> partitionEntry : entries.getValue().entrySet()) {
-                PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
-                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
-                partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
-                partitionArray.add(partitionData);
-            }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
-            topicArray.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
-        this.responseData = responseData;
-    }
-
-    public OffsetFetchResponse(Struct struct) {
-        super(struct);
-        responseData = new HashMap<TopicPartition, PartitionData>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
-            Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME);
-                String metadata = partitionResponse.getString(METADATA_KEY_NAME);
-                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
-                PartitionData partitionData = new PartitionData(offset, metadata, errorCode);
-                responseData.put(new TopicPartition(topic, partition), partitionData);
-            }
-        }
-    }
-
-    public Map<TopicPartition, PartitionData> responseData() {
-        return responseData;
-    }
-
-    public static OffsetFetchResponse parse(ByteBuffer buffer) {
-        return new OffsetFetchResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceRequest.java
deleted file mode 100644
index 55694fb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceRequest.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.Errors;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ProduceRequest  extends AbstractRequest {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id);
-    private static final String ACKS_KEY_NAME = "acks";
-    private static final String TIMEOUT_KEY_NAME = "timeout";
-    private static final String TOPIC_DATA_KEY_NAME = "topic_data";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_DATA_KEY_NAME = "data";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String RECORD_SET_KEY_NAME = "record_set";
-
-    private final short acks;
-    private final int timeout;
-    private final Map<TopicPartition, ByteBuffer> partitionRecords;
-
-    public ProduceRequest(short acks, int timeout, Map<TopicPartition, ByteBuffer> partitionRecords) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, ByteBuffer>> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords);
-        struct.set(ACKS_KEY_NAME, acks);
-        struct.set(TIMEOUT_KEY_NAME, timeout);
-        List<Struct> topicDatas = new ArrayList<Struct>(recordsByTopic.size());
-        for (Map.Entry<String, Map<Integer, ByteBuffer>> entry : recordsByTopic.entrySet()) {
-            Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, ByteBuffer> partitionEntry : entry.getValue().entrySet()) {
-                ByteBuffer buffer = partitionEntry.getValue().duplicate();
-                Struct part = topicData.instance(PARTITION_DATA_KEY_NAME)
-                                       .set(PARTITION_KEY_NAME, partitionEntry.getKey())
-                                       .set(RECORD_SET_KEY_NAME, buffer);
-                partitionArray.add(part);
-            }
-            topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray());
-            topicDatas.add(topicData);
-        }
-        struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray());
-        this.acks = acks;
-        this.timeout = timeout;
-        this.partitionRecords = partitionRecords;
-    }
-
-    public ProduceRequest(Struct struct) {
-        super(struct);
-        partitionRecords = new HashMap<TopicPartition, ByteBuffer>();
-        for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) {
-            Struct topicData = (Struct) topicDataObj;
-            String topic = topicData.getString(TOPIC_KEY_NAME);
-            for (Object partitionResponseObj : topicData.getArray(PARTITION_DATA_KEY_NAME)) {
-                Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                ByteBuffer records = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
-                partitionRecords.put(new TopicPartition(topic, partition), records);
-            }
-        }
-        acks = struct.getShort(ACKS_KEY_NAME);
-        timeout = struct.getInt(TIMEOUT_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        /* In case the producer doesn't actually want any response */
-        if (acks == 0)
-            return null;
-
-        Map<TopicPartition, ProduceResponse.PartitionResponse> responseMap = new HashMap<TopicPartition, ProduceResponse.PartitionResponse>();
-
-        for (Map.Entry<TopicPartition, ByteBuffer> entry : partitionRecords.entrySet()) {
-            responseMap.put(entry.getKey(), new ProduceResponse.PartitionResponse(Errors.forException(e).code(), ProduceResponse.INVALID_OFFSET));
-        }
-
-        switch (versionId) {
-            case 0:
-                return new ProduceResponse(responseMap);
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)));
-        }
-    }
-
-    public short acks() {
-        return acks;
-    }
-
-    public int timeout() {
-        return timeout;
-    }
-
-    public Map<TopicPartition, ByteBuffer> partitionRecords() {
-        return partitionRecords;
-    }
-
-    public static ProduceRequest parse(ByteBuffer buffer, int versionId) {
-        return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer));
-    }
-
-    public static ProduceRequest parse(ByteBuffer buffer) {
-        return new ProduceRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceResponse.java
deleted file mode 100644
index 0728d9a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ProduceResponse.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.TopicPartition;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.types.Schema;
-import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ProduceResponse extends AbstractRequestResponse {
-    
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id);
-    private static final String RESPONSES_KEY_NAME = "responses";
-
-    // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses";
-
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
-    public static final long INVALID_OFFSET = -1L;
-
-    /**
-     * Possible error code:
-     *
-     * TODO
-     */
-
-    private static final String BASE_OFFSET_KEY_NAME = "base_offset";
-
-    private final Map<TopicPartition, PartitionResponse> responses;
-
-    public ProduceResponse(Map<TopicPartition, PartitionResponse> responses) {
-        super(new Struct(CURRENT_SCHEMA));
-        Map<String, Map<Integer, PartitionResponse>> responseByTopic = CollectionUtils.groupDataByTopic(responses);
-        List<Struct> topicDatas = new ArrayList<Struct>(responseByTopic.size());
-        for (Map.Entry<String, Map<Integer, PartitionResponse>> entry : responseByTopic.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entry.getKey());
-            List<Struct> partitionArray = new ArrayList<Struct>();
-            for (Map.Entry<Integer, PartitionResponse> partitionEntry : entry.getValue().entrySet()) {
-                PartitionResponse part = partitionEntry.getValue();
-                Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME)
-                                       .set(PARTITION_KEY_NAME, partitionEntry.getKey())
-                                       .set(ERROR_CODE_KEY_NAME, part.errorCode)
-                                       .set(BASE_OFFSET_KEY_NAME, part.baseOffset);
-                partitionArray.add(partStruct);
-            }
-            topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray());
-            topicDatas.add(topicData);
-        }
-        struct.set(RESPONSES_KEY_NAME, topicDatas.toArray());
-        this.responses = responses;
-    }
-
-    public ProduceResponse(Struct struct) {
-        super(struct);
-        responses = new HashMap<TopicPartition, PartitionResponse>();
-        for (Object topicResponse : struct.getArray("responses")) {
-            Struct topicRespStruct = (Struct) topicResponse;
-            String topic = topicRespStruct.getString("topic");
-            for (Object partResponse : topicRespStruct.getArray("partition_responses")) {
-                Struct partRespStruct = (Struct) partResponse;
-                int partition = partRespStruct.getInt("partition");
-                short errorCode = partRespStruct.getShort("error_code");
-                long offset = partRespStruct.getLong("base_offset");
-                TopicPartition tp = new TopicPartition(topic, partition);
-                responses.put(tp, new PartitionResponse(errorCode, offset));
-            }
-        }
-    }
-
-    public Map<TopicPartition, PartitionResponse> responses() {
-        return this.responses;
-    }
-
-    public static final class PartitionResponse {
-        public short errorCode;
-        public long baseOffset;
-
-        public PartitionResponse(short errorCode, long baseOffset) {
-            this.errorCode = errorCode;
-            this.baseOffset = baseOffset;
-        }
-
-        @Override
-        public String toString() {
-            StringBuilder b = new StringBuilder();
-            b.append('{');
-            b.append("error: ");
-            b.append(errorCode);
-            b.append(",offset: ");
-            b.append(baseOffset);
-            b.append('}');
-            return b.toString();
-        }
-    }
-
-    public static ProduceResponse parse(ByteBuffer buffer) {
-        return new ProduceResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestHeader.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestHeader.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestHeader.java
deleted file mode 100644
index 82ef7c7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestHeader.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
-import org.apache.flink.kafka_backport.common.protocol.Protocol;
-import org.apache.flink.kafka_backport.common.protocol.types.Field;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The header for a request in the Kafka protocol
- */
-public class RequestHeader extends AbstractRequestResponse {
-
-    private static final Field API_KEY_FIELD = Protocol.REQUEST_HEADER.get("api_key");
-    private static final Field API_VERSION_FIELD = Protocol.REQUEST_HEADER.get("api_version");
-    private static final Field CLIENT_ID_FIELD = Protocol.REQUEST_HEADER.get("client_id");
-    private static final Field CORRELATION_ID_FIELD = Protocol.REQUEST_HEADER.get("correlation_id");
-
-    private final short apiKey;
-    private final short apiVersion;
-    private final String clientId;
-    private final int correlationId;
-
-    public RequestHeader(Struct header) {
-        super(header);
-        apiKey = struct.getShort(API_KEY_FIELD);
-        apiVersion = struct.getShort(API_VERSION_FIELD);
-        clientId = struct.getString(CLIENT_ID_FIELD);
-        correlationId = struct.getInt(CORRELATION_ID_FIELD);
-    }
-
-    public RequestHeader(short apiKey, String client, int correlation) {
-        this(apiKey, ProtoUtils.latestVersion(apiKey), client, correlation);
-    }
-
-    public RequestHeader(short apiKey, short version, String client, int correlation) {
-        super(new Struct(Protocol.REQUEST_HEADER));
-        struct.set(API_KEY_FIELD, apiKey);
-        struct.set(API_VERSION_FIELD, version);
-        struct.set(CLIENT_ID_FIELD, client);
-        struct.set(CORRELATION_ID_FIELD, correlation);
-        this.apiKey = apiKey;
-        this.apiVersion = version;
-        this.clientId = client;
-        this.correlationId = correlation;
-    }
-
-    public short apiKey() {
-        return apiKey;
-    }
-
-    public short apiVersion() {
-        return apiVersion;
-    }
-
-    public String clientId() {
-        return clientId;
-    }
-
-    public int correlationId() {
-        return correlationId;
-    }
-
-    public static RequestHeader parse(ByteBuffer buffer) {
-        return new RequestHeader((Struct) Protocol.REQUEST_HEADER.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestSend.java
deleted file mode 100644
index 1815005..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/RequestSend.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.network.NetworkSend;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A send object for a kafka request
- */
-public class RequestSend extends NetworkSend {
-
-    private final RequestHeader header;
-    private final Struct body;
-
-    public RequestSend(String destination, RequestHeader header, Struct body) {
-        super(destination, serialize(header, body));
-        this.header = header;
-        this.body = body;
-    }
-
-    private static ByteBuffer serialize(RequestHeader header, Struct body) {
-        ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
-        header.writeTo(buffer);
-        body.writeTo(buffer);
-        buffer.rewind();
-        return buffer;
-    }
-
-    public RequestHeader header() {
-        return this.header;
-    }
-
-    public Struct body() {
-        return body;
-    }
-
-    @Override
-    public String toString() {
-        return "RequestSend(header=" + header.toString() + ", body=" + body.toString() + ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseHeader.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseHeader.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseHeader.java
deleted file mode 100644
index a6aad5e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseHeader.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.protocol.Protocol;
-import org.apache.flink.kafka_backport.common.protocol.types.Field;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A response header in the kafka protocol.
- */
-public class ResponseHeader extends AbstractRequestResponse {
-
-    private static final Field CORRELATION_KEY_FIELD = Protocol.RESPONSE_HEADER.get("correlation_id");
-
-    private final int correlationId;
-
-    public ResponseHeader(Struct header) {
-        super(header);
-        correlationId = struct.getInt(CORRELATION_KEY_FIELD);
-    }
-
-    public ResponseHeader(int correlationId) {
-        super(new Struct(Protocol.RESPONSE_HEADER));
-        struct.set(CORRELATION_KEY_FIELD, correlationId);
-        this.correlationId = correlationId;
-    }
-
-    public int correlationId() {
-        return correlationId;
-    }
-
-    public static ResponseHeader parse(ByteBuffer buffer) {
-        return new ResponseHeader((Struct) Protocol.RESPONSE_HEADER.read(buffer));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseSend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseSend.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseSend.java
deleted file mode 100644
index ee3b393..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ResponseSend.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.kafka_backport.common.requests;
-
-import org.apache.flink.kafka_backport.common.network.NetworkSend;
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ResponseSend extends NetworkSend {
-
-    public ResponseSend(String destination, ResponseHeader header, Struct body) {
-        super(destination, serialize(header, body));
-    }
-
-    public ResponseSend(String destination, ResponseHeader header, AbstractRequestResponse response) {
-        this(destination, header, response.toStruct());
-    }
-
-    private static ByteBuffer serialize(ResponseHeader header, Struct body) {
-        ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
-        header.writeTo(buffer);
-        body.writeTo(buffer);
-        buffer.rewind();
-        return buffer;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArrayDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArrayDeserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArrayDeserializer.java
deleted file mode 100644
index 7e0b3e9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArrayDeserializer.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.kafka_backport.common.serialization;
-
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ByteArrayDeserializer implements Deserializer<byte[]> {
-
-    @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        // nothing to do
-    }
-
-    @Override
-    public byte[] deserialize(String topic, byte[] data) {
-        return data;
-    }
-
-    @Override
-    public void close() {
-        // nothing to do
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArraySerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArraySerializer.java
deleted file mode 100644
index f835375..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/ByteArraySerializer.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.kafka_backport.common.serialization;
-
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ByteArraySerializer implements Serializer<byte[]> {
-
-    @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        // nothing to do
-    }
-
-    @Override
-    public byte[] serialize(String topic, byte[] data) {
-        return data;
-    }
-
-    @Override
-    public void close() {
-        // nothing to do
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Deserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Deserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Deserializer.java
deleted file mode 100644
index 4d2de90..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Deserializer.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.kafka_backport.common.serialization;
-
-import java.io.Closeable;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- *
- * @param <T> Type to be deserialized into.
- *
- * A class that implements this interface is expected to have a constructor with no parameter.
- */
-public interface Deserializer<T> extends Closeable {
-
-    /**
-     * Configure this class.
-     * @param configs configs in key/value pairs
-     * @param isKey whether is for key or value
-     */
-    public void configure(Map<String, ?> configs, boolean isKey);
-    
-    /**
-     *
-     * @param topic topic associated with the data
-     * @param data serialized bytes
-     * @return deserialized typed data
-     */
-    public T deserialize(String topic, byte[] data);
-
-    @Override
-    public void close();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerDeserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerDeserializer.java
deleted file mode 100644
index c5833d5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerDeserializer.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.kafka_backport.common.serialization;
-
-import org.apache.flink.kafka_backport.common.errors.SerializationException;
-
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class IntegerDeserializer implements Deserializer<Integer> {
-
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        // nothing to do
-    }
-
-    public Integer deserialize(String topic, byte[] data) {
-        if (data == null)
-            return null;
-        if (data.length != 4) {
-            throw new SerializationException("Size of data received by IntegerDeserializer is " +
-                    "not 4");
-        }
-
-        int value = 0;
-        for (byte b : data) {
-            value <<= 8;
-            value |= b & 0xFF;
-        }
-        return value;
-    }
-
-    public void close() {
-        // nothing to do
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerSerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerSerializer.java
deleted file mode 100644
index dcbd7be..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/IntegerSerializer.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.kafka_backport.common.serialization;
-
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class IntegerSerializer implements Serializer<Integer> {
-
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        // nothing to do
-    }
-
-    public byte[] serialize(String topic, Integer data) {
-        if (data == null)
-            return null;
-
-        return new byte[] {
-            (byte) (data >>> 24),
-            (byte) (data >>> 16),
-            (byte) (data >>> 8),
-            data.byteValue()
-        };
-    }
-
-    public void close() {
-        // nothing to do
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Serializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Serializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Serializer.java
deleted file mode 100644
index 1725e36..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/Serializer.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-
-package org.apache.flink.kafka_backport.common.serialization;
-
-import java.io.Closeable;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- *
- * @param <T> Type to be serialized from.
- *
- * A class that implements this interface is expected to have a constructor with no parameter.
- */
-public interface Serializer<T> extends Closeable {
-
-    /**
-     * Configure this class.
-     * @param configs configs in key/value pairs
-     * @param isKey whether is for key or value
-     */
-    public void configure(Map<String, ?> configs, boolean isKey);
-
-    /**
-     * @param topic topic associated with data
-     * @param data typed data
-     * @return serialized bytes
-     */
-    public byte[] serialize(String topic, T data);
-
-
-    /**
-     * Close this serializer.
-     * This method has to be idempotent if the serializer is used in KafkaProducer because it might be called
-     * multiple times.
-     */
-    @Override
-    public void close();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringDeserializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringDeserializer.java
deleted file mode 100644
index 4d2ed4c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringDeserializer.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.kafka_backport.common.serialization;
-
-import org.apache.flink.kafka_backport.common.errors.SerializationException;
-
-import java.io.UnsupportedEncodingException;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- *  String encoding defaults to UTF8 and can be customized by setting the property key.deserializer.encoding,
- *  value.deserializer.encoding or deserializer.encoding. The first two take precedence over the last.
- */
-public class StringDeserializer implements Deserializer<String> {
-    private String encoding = "UTF8";
-
-    @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        String propertyName = isKey ? "key.deserializer.encoding" : "value.deserializer.encoding";
-        Object encodingValue = configs.get(propertyName);
-        if (encodingValue == null)
-            encodingValue = configs.get("deserializer.encoding");
-        if (encodingValue != null && encodingValue instanceof String)
-            encoding = (String) encodingValue;
-    }
-
-    @Override
-    public String deserialize(String topic, byte[] data) {
-        try {
-            if (data == null)
-                return null;
-            else
-                return new String(data, encoding);
-        } catch (UnsupportedEncodingException e) {
-            throw new SerializationException("Error when deserializing byte[] to string due to unsupported encoding " + encoding);
-        }
-    }
-
-    @Override
-    public void close() {
-        // nothing to do
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringSerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringSerializer.java
deleted file mode 100644
index fae4c21..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/serialization/StringSerializer.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.kafka_backport.common.serialization;
-
-import org.apache.flink.kafka_backport.common.errors.SerializationException;
-
-import java.io.UnsupportedEncodingException;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- *  String encoding defaults to UTF8 and can be customized by setting the property key.serializer.encoding,
- *  value.serializer.encoding or serializer.encoding. The first two take precedence over the last.
- */
-public class StringSerializer implements Serializer<String> {
-    private String encoding = "UTF8";
-
-    @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
-        String propertyName = isKey ? "key.serializer.encoding" : "value.serializer.encoding";
-        Object encodingValue = configs.get(propertyName);
-        if (encodingValue == null)
-            encodingValue = configs.get("serializer.encoding");
-        if (encodingValue != null && encodingValue instanceof String)
-            encoding = (String) encodingValue;
-    }
-
-    @Override
-    public byte[] serialize(String topic, String data) {
-        try {
-            if (data == null)
-                return null;
-            else
-                return data.getBytes(encoding);
-        } catch (UnsupportedEncodingException e) {
-            throw new SerializationException("Error when serializing string to byte[] due to unsupported encoding " + encoding);
-        }
-    }
-
-    @Override
-    public void close() {
-        // nothing to do
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/AbstractIterator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/AbstractIterator.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/AbstractIterator.java
deleted file mode 100644
index 2af94b6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/AbstractIterator.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.kafka_backport.common.utils;
-
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A base class that simplifies implementing an iterator
- * @param <T> The type of thing we are iterating over
- */
-public abstract class AbstractIterator<T> implements Iterator<T> {
-
-    private static enum State {
-        READY, NOT_READY, DONE, FAILED
-    };
-
-    private State state = State.NOT_READY;
-    private T next;
-
-    @Override
-    public boolean hasNext() {
-        switch (state) {
-            case FAILED:
-                throw new IllegalStateException("Iterator is in failed state");
-            case DONE:
-                return false;
-            case READY:
-                return true;
-            default:
-                return maybeComputeNext();
-        }
-    }
-
-    @Override
-    public T next() {
-        if (!hasNext())
-            throw new NoSuchElementException();
-        state = State.NOT_READY;
-        if (next == null)
-            throw new IllegalStateException("Expected item but none found.");
-        return next;
-    }
-
-    @Override
-    public void remove() {
-        throw new UnsupportedOperationException("Removal not supported");
-    }
-
-    public T peek() {
-        if (!hasNext())
-            throw new NoSuchElementException();
-        return next;
-    }
-
-    protected T allDone() {
-        state = State.DONE;
-        return null;
-    }
-
-    protected abstract T makeNext();
-
-    private Boolean maybeComputeNext() {
-        state = State.FAILED;
-        next = makeNext();
-        if (state == State.DONE) {
-            return false;
-        } else {
-            state = State.READY;
-            return true;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CollectionUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CollectionUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CollectionUtils.java
deleted file mode 100644
index 7960331..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CollectionUtils.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.kafka_backport.common.utils;
-
-import org.apache.flink.kafka_backport.common.TopicPartition;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class CollectionUtils {
-    /**
-     * group data by topic
-     * @param data Data to be partitioned
-     * @param <T> Partition data type
-     * @return partitioned data
-     */
-    public static <T> Map<String, Map<Integer, T>> groupDataByTopic(Map<TopicPartition, T> data) {
-        Map<String, Map<Integer, T>> dataByTopic = new HashMap<String, Map<Integer, T>>();
-        for (Map.Entry<TopicPartition, T> entry: data.entrySet()) {
-            String topic = entry.getKey().topic();
-            int partition = entry.getKey().partition();
-            Map<Integer, T> topicData = dataByTopic.get(topic);
-            if (topicData == null) {
-                topicData = new HashMap<Integer, T>();
-                dataByTopic.put(topic, topicData);
-            }
-            topicData.put(partition, entry.getValue());
-        }
-        return dataByTopic;
-    }
-
-    /**
-     * group partitions by topic
-     * @param partitions
-     * @return partitions per topic
-     */
-    public static Map<String, List<Integer>> groupDataByTopic(List<TopicPartition> partitions) {
-        Map<String, List<Integer>> partitionsByTopic = new HashMap<String, List<Integer>>();
-        for (TopicPartition tp: partitions) {
-            String topic = tp.topic();
-            List<Integer> topicData = partitionsByTopic.get(topic);
-            if (topicData == null) {
-                topicData = new ArrayList<Integer>();
-                partitionsByTopic.put(topic, topicData);
-            }
-            topicData.add(tp.partition());
-        }
-        return  partitionsByTopic;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CopyOnWriteMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CopyOnWriteMap.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CopyOnWriteMap.java
deleted file mode 100644
index dcf219a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/utils/CopyOnWriteMap.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.kafka_backport.common.utils;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A simple read-optimized map implementation that synchronizes only writes and does a full copy on each modification
- */
-public class CopyOnWriteMap<K, V> implements ConcurrentMap<K, V> {
-
-    private volatile Map<K, V> map;
-
-    public CopyOnWriteMap() {
-        this.map = Collections.emptyMap();
-    }
-
-    public CopyOnWriteMap(Map<K, V> map) {
-        this.map = Collections.unmodifiableMap(map);
-    }
-
-    @Override
-    public boolean containsKey(Object k) {
-        return map.containsKey(k);
-    }
-
-    @Override
-    public boolean containsValue(Object v) {
-        return map.containsValue(v);
-    }
-
-    @Override
-    public Set<Entry<K, V>> entrySet() {
-        return map.entrySet();
-    }
-
-    @Override
-    public V get(Object k) {
-        return map.get(k);
-    }
-
-    @Override
-    public boolean isEmpty() {
-        return map.isEmpty();
-    }
-
-    @Override
-    public Set<K> keySet() {
-        return map.keySet();
-    }
-
-    @Override
-    public int size() {
-        return map.size();
-    }
-
-    @Override
-    public Collection<V> values() {
-        return map.values();
-    }
-
-    @Override
-    public synchronized void clear() {
-        this.map = Collections.emptyMap();
-    }
-
-    @Override
-    public synchronized V put(K k, V v) {
-        Map<K, V> copy = new HashMap<K, V>(this.map);
-        V prev = copy.put(k, v);
-        this.map = Collections.unmodifiableMap(copy);
-        return prev;
-    }
-
-    @Override
-    public synchronized void putAll(Map<? extends K, ? extends V> entries) {
-        Map<K, V> copy = new HashMap<K, V>(this.map);
-        copy.putAll(entries);
-        this.map = Collections.unmodifiableMap(copy);
-    }
-
-    @Override
-    public synchronized V remove(Object key) {
-        Map<K, V> copy = new HashMap<K, V>(this.map);
-        V prev = copy.remove(key);
-        this.map = Collections.unmodifiableMap(copy);
-        return prev;
-    }
-
-    @Override
-    public synchronized V putIfAbsent(K k, V v) {
-        if (!containsKey(k))
-            return put(k, v);
-        else
-            return get(k);
-    }
-
-    @Override
-    public synchronized boolean remove(Object k, Object v) {
-        if (containsKey(k) && get(k).equals(v)) {
-            remove(k);
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public synchronized boolean replace(K k, V original, V replacement) {
-        if (containsKey(k) && get(k).equals(original)) {
-            put(k, replacement);
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public synchronized V replace(K k, V v) {
-        if (containsKey(k)) {
-            return put(k, v);
-        } else {
-            return null;
-        }
-    }
-
-}


[35/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ThrottledMapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ThrottledMapper.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ThrottledMapper.java
deleted file mode 100644
index 872d42f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/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.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/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.java
deleted file mode 100644
index 1e5f027..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/Tuple2Partitioner.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.testutils;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.connectors.SerializableKafkaPartitioner;
-
-/**
- * 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 implements SerializableKafkaPartitioner {
-	
-	private static final long serialVersionUID = 1L;
-
-	private final int expectedPartitions;
-
-	
-	public Tuple2Partitioner(int expectedPartitions) {
-		this.expectedPartitions = expectedPartitions;
-	}
-
-	@Override
-	public int partition(Object key, int numPartitions) {
-		if (numPartitions != expectedPartitions) {
-			throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions");
-		}
-		@SuppressWarnings("unchecked")
-		Tuple2<Integer, Integer> element = (Tuple2<Integer, Integer>) key;
-		
-		return element.f0;
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ValidatingExactlyOnceSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ValidatingExactlyOnceSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ValidatingExactlyOnceSink.java
deleted file mode 100644
index a3c35fc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/java/org/apache/flink/streaming/connectors/testutils/ValidatingExactlyOnceSink.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.testutils;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-
-import java.util.BitSet;
-
-public class ValidatingExactlyOnceSink implements SinkFunction<Integer>, Checkpointed<Tuple2<Integer, BitSet>> {
-
-	private static final long serialVersionUID = 1748426382527469932L;
-	
-	private final int numElementsTotal;
-	
-	private BitSet duplicateChecker = new BitSet();  // this is checkpointed
-
-	private int numElements; // this is checkpointed
-
-	
-	public ValidatingExactlyOnceSink(int numElementsTotal) {
-		this.numElementsTotal = numElementsTotal;
-	}
-
-	
-	@Override
-	public void invoke(Integer value) throws Exception {
-		numElements++;
-		
-		if (duplicateChecker.get(value)) {
-			throw new Exception("Received a duplicate");
-		}
-		duplicateChecker.set(value);
-		
-		if (numElements == numElementsTotal) {
-			// validate
-			if (duplicateChecker.cardinality() != numElementsTotal) {
-				throw new Exception("Duplicate checker has wrong cardinality");
-			}
-			else if (duplicateChecker.nextClearBit(0) != numElementsTotal) {
-				throw new Exception("Received sparse sequence");
-			}
-			else {
-				throw new SuccessException();
-			}
-		}
-	}
-
-	@Override
-	public Tuple2<Integer, BitSet> snapshotState(long checkpointId, long checkpointTimestamp) {
-		return new Tuple2<>(numElements, duplicateChecker);
-	}
-
-	@Override
-	public void restoreState(Tuple2<Integer, BitSet> state) {
-		this.numElements = state.f0;
-		this.duplicateChecker = state.f1;
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/resources/logback-test.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/test/resources/logback-test.xml
deleted file mode 100644
index 45b3b92..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/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/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/pom.xml
index c369c61..bbbd2be 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/pom.xml
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/pom.xml
@@ -96,6 +96,14 @@ under the License.
 			<groupId>org.apache.curator</groupId>
 			<artifactId>curator-test</artifactId>
 			<version>2.7.1</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
 		</dependency>
 
 	</dependencies>

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/KafkaSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/KafkaSink.java
new file mode 100644
index 0000000..6d39541
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/KafkaSink.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors;
+
+import com.google.common.base.Preconditions;
+import kafka.javaapi.producer.Producer;
+import kafka.producer.KeyedMessage;
+import kafka.producer.ProducerConfig;
+import kafka.serializer.DefaultEncoder;
+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.util.serialization.SerializationSchema;
+import org.apache.flink.util.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Properties;
+
+
+/**
+ * Sink that emits its inputs to a Kafka topic.
+ *
+ * @param <IN>
+ * 		Type of the sink input
+ */
+public class KafkaSink<IN> extends RichSinkFunction<IN> {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(KafkaSink.class);
+
+	private Producer<IN, byte[]> producer;
+	private Properties userDefinedProperties;
+	private String topicId;
+	private String brokerList;
+	private SerializationSchema<IN, byte[]> schema;
+	private SerializableKafkaPartitioner partitioner;
+	private Class<? extends SerializableKafkaPartitioner> partitionerClass = null;
+
+	/**
+	 * Creates a KafkaSink for a given topic. The sink produces its input to
+	 * the topic.
+	 *
+	 * @param brokerList
+	 *			Addresses of the brokers
+	 * @param topicId
+	 * 		ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 		User defined serialization schema.
+	 */
+	public KafkaSink(String brokerList, String topicId, SerializationSchema<IN, byte[]> serializationSchema) {
+		this(brokerList, topicId, new Properties(), serializationSchema);
+	}
+
+	/**
+	 * Creates a KafkaSink for a given topic with custom Producer configuration.
+	 * If you use this constructor, the broker should be set with the "metadata.broker.list"
+	 * configuration.
+	 *
+	 * @param brokerList
+	 * 		Addresses of the brokers
+	 * @param topicId
+	 * 		ID of the Kafka topic.
+	 * @param producerConfig
+	 * 		Configurations of the Kafka producer
+	 * @param serializationSchema
+	 * 		User defined serialization schema.
+	 */
+	public KafkaSink(String brokerList, String topicId, Properties producerConfig, SerializationSchema<IN, byte[]> serializationSchema) {
+		String[] elements = brokerList.split(",");
+		for(String broker: elements) {
+			NetUtils.getCorrectHostnamePort(broker);
+		}
+		Preconditions.checkNotNull(topicId, "TopicID not set");
+
+		this.brokerList = brokerList;
+		this.topicId = topicId;
+		this.schema = serializationSchema;
+		this.partitionerClass = null;
+		this.userDefinedProperties = producerConfig;
+	}
+
+	/**
+	 * Creates a KafkaSink for a given topic. The sink produces its input to
+	 * the topic.
+	 *
+	 * @param brokerList
+	 * @param topicId
+	 * 		ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 		User defined serialization schema.
+	 * @param partitioner
+	 * 		User defined partitioner.
+	 */
+	public KafkaSink(String brokerList, String topicId, SerializationSchema<IN, byte[]> serializationSchema, SerializableKafkaPartitioner partitioner) {
+		this(brokerList, topicId, serializationSchema);
+		ClosureCleaner.ensureSerializable(partitioner);
+		this.partitioner = partitioner;
+	}
+
+	public KafkaSink(String brokerList,
+					String topicId,
+					SerializationSchema<IN, byte[]> serializationSchema,
+					Class<? extends SerializableKafkaPartitioner> partitioner) {
+		this(brokerList, topicId, serializationSchema);
+		this.partitionerClass = partitioner;
+	}
+
+	/**
+	 * Initializes the connection to Kafka.
+	 */
+	@Override
+	public void open(Configuration configuration) {
+
+		Properties properties = new Properties();
+
+		properties.put("metadata.broker.list", brokerList);
+		properties.put("request.required.acks", "-1");
+		properties.put("message.send.max.retries", "10");
+
+		properties.put("serializer.class", DefaultEncoder.class.getCanonicalName());
+
+		// this will not be used as the key will not be serialized
+		properties.put("key.serializer.class", DefaultEncoder.class.getCanonicalName());
+
+		for (Map.Entry<Object, Object> propertiesEntry : userDefinedProperties.entrySet()) {
+			properties.put(propertiesEntry.getKey(), propertiesEntry.getValue());
+		}
+
+		if (partitioner != null) {
+			properties.put("partitioner.class", PartitionerWrapper.class.getCanonicalName());
+			// java serialization will do the rest.
+			properties.put(PartitionerWrapper.SERIALIZED_WRAPPER_NAME, partitioner);
+		}
+		if (partitionerClass != null) {
+			properties.put("partitioner.class", partitionerClass);
+		}
+
+		ProducerConfig config = new ProducerConfig(properties);
+
+		try {
+			producer = new Producer<IN, byte[]>(config);
+		} catch (NullPointerException e) {
+			throw new RuntimeException("Cannot connect to Kafka broker " + brokerList, e);
+		}
+	}
+
+	/**
+	 * Called when new data arrives to the sink, and forwards it to Kafka.
+	 *
+	 * @param next
+	 * 		The incoming data
+	 */
+	@Override
+	public void invoke(IN next) {
+		byte[] serialized = schema.serialize(next);
+
+		// Sending message without serializable key.
+		producer.send(new KeyedMessage<IN, byte[]>(topicId, null, next, serialized));
+	}
+
+	@Override
+	public void close() {
+		if (producer != null) {
+			producer.close();
+		}
+	}
+
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/SerializableKafkaPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/SerializableKafkaPartitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/SerializableKafkaPartitioner.java
new file mode 100644
index 0000000..aff0c3a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/SerializableKafkaPartitioner.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;
+
+import kafka.producer.Partitioner;
+
+import java.io.Serializable;
+
+public interface SerializableKafkaPartitioner extends Serializable, Partitioner {
+
+}

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


[14/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchRequest.java
new file mode 100644
index 0000000..f797ebe
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchRequest.java
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class FetchRequest extends AbstractRequest {
+    
+    public static final int CONSUMER_REPLICA_ID = -1;
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id);
+    private static final String REPLICA_ID_KEY_NAME = "replica_id";
+    private static final String MAX_WAIT_KEY_NAME = "max_wait_time";
+    private static final String MIN_BYTES_KEY_NAME = "min_bytes";
+    private static final String TOPICS_KEY_NAME = "topics";
+
+    // topic level field names
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partitions";
+
+    // partition level field names
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset";
+    private static final String MAX_BYTES_KEY_NAME = "max_bytes";
+
+    private final int replicaId;
+    private final int maxWait;
+    private final int minBytes;
+    private final Map<TopicPartition, PartitionData> fetchData;
+
+    public static final class PartitionData {
+        public final long offset;
+        public final int maxBytes;
+
+        public PartitionData(long offset, int maxBytes) {
+            this.offset = offset;
+            this.maxBytes = maxBytes;
+        }
+    }
+
+    /**
+     * Create a non-replica fetch request
+     */
+    public FetchRequest(int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
+        this(CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData);
+    }
+
+    /**
+     * Create a replica fetch request
+     */
+    public FetchRequest(int replicaId, int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
+        super(new Struct(CURRENT_SCHEMA));
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(fetchData);
+
+        struct.set(REPLICA_ID_KEY_NAME, replicaId);
+        struct.set(MAX_WAIT_KEY_NAME, maxWait);
+        struct.set(MIN_BYTES_KEY_NAME, minBytes);
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry : topicsData.entrySet()) {
+            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData fetchPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset);
+                partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+        this.replicaId = replicaId;
+        this.maxWait = maxWait;
+        this.minBytes = minBytes;
+        this.fetchData = fetchData;
+    }
+
+    public FetchRequest(Struct struct) {
+        super(struct);
+        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
+        maxWait = struct.getInt(MAX_WAIT_KEY_NAME);
+        minBytes = struct.getInt(MIN_BYTES_KEY_NAME);
+        fetchData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                long offset = partitionResponse.getLong(FETCH_OFFSET_KEY_NAME);
+                int maxBytes = partitionResponse.getInt(MAX_BYTES_KEY_NAME);
+                PartitionData partitionData = new PartitionData(offset, maxBytes);
+                fetchData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        Map<TopicPartition, FetchResponse.PartitionData> responseData = new HashMap<TopicPartition, FetchResponse.PartitionData>();
+
+        for (Map.Entry<TopicPartition, PartitionData> entry: fetchData.entrySet()) {
+            FetchResponse.PartitionData partitionResponse = new FetchResponse.PartitionData(Errors.forException(e).code(),
+                    FetchResponse.INVALID_HIGHWATERMARK,
+                    FetchResponse.EMPTY_RECORD_SET);
+            responseData.put(entry.getKey(), partitionResponse);
+        }
+
+        switch (versionId) {
+            case 0:
+                return new FetchResponse(responseData);
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.FETCH.id)));
+        }
+    }
+
+    public int replicaId() {
+        return replicaId;
+    }
+
+    public int maxWait() {
+        return maxWait;
+    }
+
+    public int minBytes() {
+        return minBytes;
+    }
+
+    public Map<TopicPartition, PartitionData> fetchData() {
+        return fetchData;
+    }
+
+    public static FetchRequest parse(ByteBuffer buffer, int versionId) {
+        return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer));
+    }
+
+    public static FetchRequest parse(ByteBuffer buffer) {
+        return new FetchRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchResponse.java
new file mode 100644
index 0000000..158833e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/FetchResponse.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class FetchResponse extends AbstractRequestResponse {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id);
+    private static final String RESPONSES_KEY_NAME = "responses";
+
+    // topic level field names
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partition_responses";
+
+    // partition level field names
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    /**
+     * Possible error code:
+     *
+     *  OFFSET_OUT_OF_RANGE (1)
+     *  UNKNOWN_TOPIC_OR_PARTITION (3)
+     *  NOT_LEADER_FOR_PARTITION (6)
+     *  REPLICA_NOT_AVAILABLE (9)
+     *  UNKNOWN (-1)
+     */
+
+    private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark";
+    private static final String RECORD_SET_KEY_NAME = "record_set";
+
+    public static final long INVALID_HIGHWATERMARK = -1L;
+    public static final ByteBuffer EMPTY_RECORD_SET = ByteBuffer.allocate(0);
+
+    private final Map<TopicPartition, PartitionData> responseData;
+
+    public static final class PartitionData {
+        public final short errorCode;
+        public final long highWatermark;
+        public final ByteBuffer recordSet;
+
+        public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) {
+            this.errorCode = errorCode;
+            this.highWatermark = highWatermark;
+            this.recordSet = recordSet;
+        }
+    }
+
+    public FetchResponse(Map<TopicPartition, PartitionData> responseData) {
+        super(new Struct(CURRENT_SCHEMA));
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData fetchPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
+                partitionData.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark);
+                partitionData.set(RECORD_SET_KEY_NAME, fetchPartitionData.recordSet);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        this.responseData = responseData;
+    }
+
+    public FetchResponse(Struct struct) {
+        super(struct);
+        responseData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+                long highWatermark = partitionResponse.getLong(HIGH_WATERMARK_KEY_NAME);
+                ByteBuffer recordSet = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
+                PartitionData partitionData = new PartitionData(errorCode, highWatermark, recordSet);
+                responseData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    public Map<TopicPartition, PartitionData> responseData() {
+        return responseData;
+    }
+
+    public static FetchResponse parse(ByteBuffer buffer) {
+        return new FetchResponse((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatRequest.java
new file mode 100644
index 0000000..c8abb67
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatRequest.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class HeartbeatRequest extends AbstractRequest {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id";
+    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
+
+    private final String groupId;
+    private final int groupGenerationId;
+    private final String consumerId;
+
+    public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) {
+        super(new Struct(CURRENT_SCHEMA));
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId);
+        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        this.groupId = groupId;
+        this.groupGenerationId = groupGenerationId;
+        this.consumerId = consumerId;
+    }
+
+    public HeartbeatRequest(Struct struct) {
+        super(struct);
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+        groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME);
+        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        switch (versionId) {
+            case 0:
+                return new HeartbeatResponse(Errors.forException(e).code());
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.HEARTBEAT.id)));
+        }
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public int groupGenerationId() {
+        return groupGenerationId;
+    }
+
+    public String consumerId() {
+        return consumerId;
+    }
+
+    public static HeartbeatRequest parse(ByteBuffer buffer, int versionId) {
+        return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer));
+    }
+
+    public static HeartbeatRequest parse(ByteBuffer buffer) {
+        return new HeartbeatRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatResponse.java
new file mode 100644
index 0000000..4bf6669
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/HeartbeatResponse.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class HeartbeatResponse extends AbstractRequestResponse {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id);
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    /**
+     * Possible error code:
+     *
+     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
+     * NOT_COORDINATOR_FOR_CONSUMER (16)
+     * ILLEGAL_GENERATION (22)
+     * UNKNOWN_CONSUMER_ID (25)
+     */
+
+    private final short errorCode;
+    public HeartbeatResponse(short errorCode) {
+        super(new Struct(CURRENT_SCHEMA));
+        struct.set(ERROR_CODE_KEY_NAME, errorCode);
+        this.errorCode = errorCode;
+    }
+
+    public HeartbeatResponse(Struct struct) {
+        super(struct);
+        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+    }
+
+    public short errorCode() {
+        return errorCode;
+    }
+
+    public static HeartbeatResponse parse(ByteBuffer buffer) {
+        return new HeartbeatResponse((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupRequest.java
new file mode 100644
index 0000000..f098d18
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupRequest.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class JoinGroupRequest extends AbstractRequest {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String SESSION_TIMEOUT_KEY_NAME = "session_timeout";
+    private static final String TOPICS_KEY_NAME = "topics";
+    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
+    private static final String STRATEGY_KEY_NAME = "partition_assignment_strategy";
+
+    public static final String UNKNOWN_CONSUMER_ID = "";
+
+    private final String groupId;
+    private final int sessionTimeout;
+    private final List<String> topics;
+    private final String consumerId;
+    private final String strategy;
+
+    public JoinGroupRequest(String groupId, int sessionTimeout, List<String> topics, String consumerId, String strategy) {
+        super(new Struct(CURRENT_SCHEMA));
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout);
+        struct.set(TOPICS_KEY_NAME, topics.toArray());
+        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        struct.set(STRATEGY_KEY_NAME, strategy);
+        this.groupId = groupId;
+        this.sessionTimeout = sessionTimeout;
+        this.topics = topics;
+        this.consumerId = consumerId;
+        this.strategy = strategy;
+    }
+
+    public JoinGroupRequest(Struct struct) {
+        super(struct);
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+        sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME);
+        Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME);
+        topics = new ArrayList<String>();
+        for (Object topic: topicsArray)
+            topics.add((String) topic);
+        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+        strategy = struct.getString(STRATEGY_KEY_NAME);
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        switch (versionId) {
+            case 0:
+                return new JoinGroupResponse(
+                        Errors.forException(e).code(),
+                        JoinGroupResponse.UNKNOWN_GENERATION_ID,
+                        JoinGroupResponse.UNKNOWN_CONSUMER_ID,
+                        Collections.<TopicPartition>emptyList());
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id)));
+        }
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public int sessionTimeout() {
+        return sessionTimeout;
+    }
+
+    public List<String> topics() {
+        return topics;
+    }
+
+    public String consumerId() {
+        return consumerId;
+    }
+
+    public String strategy() {
+        return strategy;
+    }
+
+    public static JoinGroupRequest parse(ByteBuffer buffer, int versionId) {
+        return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, versionId, buffer));
+    }
+
+    public static JoinGroupRequest parse(ByteBuffer buffer) {
+        return new JoinGroupRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupResponse.java
new file mode 100644
index 0000000..7d9b647
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/JoinGroupResponse.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class JoinGroupResponse extends AbstractRequestResponse {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id);
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    /**
+     * Possible error code:
+     *
+     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
+     * NOT_COORDINATOR_FOR_CONSUMER (16)
+     * INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY (23)
+     * UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY (24)
+     * UNKNOWN_CONSUMER_ID (25)
+     * INVALID_SESSION_TIMEOUT (26)
+     */
+
+    private static final String GENERATION_ID_KEY_NAME = "group_generation_id";
+    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
+    private static final String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partitions";
+
+    public static final int UNKNOWN_GENERATION_ID = -1;
+    public static final String UNKNOWN_CONSUMER_ID = "";
+
+    private final short errorCode;
+    private final int generationId;
+    private final String consumerId;
+    private final List<TopicPartition> assignedPartitions;
+
+    public JoinGroupResponse(short errorCode, int generationId, String consumerId, List<TopicPartition> assignedPartitions) {
+        super(new Struct(CURRENT_SCHEMA));
+
+        Map<String, List<Integer>> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions);
+
+        struct.set(ERROR_CODE_KEY_NAME, errorCode);
+        struct.set(GENERATION_ID_KEY_NAME, generationId);
+        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, List<Integer>> entries: partitionsByTopic.entrySet()) {
+            Struct topicData = struct.instance(ASSIGNED_PARTITIONS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            topicData.set(PARTITIONS_KEY_NAME, entries.getValue().toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(ASSIGNED_PARTITIONS_KEY_NAME, topicArray.toArray());
+
+        this.errorCode = errorCode;
+        this.generationId = generationId;
+        this.consumerId = consumerId;
+        this.assignedPartitions = assignedPartitions;
+    }
+
+    public JoinGroupResponse(Struct struct) {
+        super(struct);
+        assignedPartitions = new ArrayList<TopicPartition>();
+        for (Object topicDataObj : struct.getArray(ASSIGNED_PARTITIONS_KEY_NAME)) {
+            Struct topicData = (Struct) topicDataObj;
+            String topic = topicData.getString(TOPIC_KEY_NAME);
+            for (Object partitionObj : topicData.getArray(PARTITIONS_KEY_NAME))
+                assignedPartitions.add(new TopicPartition(topic, (Integer) partitionObj));
+        }
+        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+        generationId = struct.getInt(GENERATION_ID_KEY_NAME);
+        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+    }
+
+    public short errorCode() {
+        return errorCode;
+    }
+
+    public int generationId() {
+        return generationId;
+    }
+
+    public String consumerId() {
+        return consumerId;
+    }
+
+    public List<TopicPartition> assignedPartitions() {
+        return assignedPartitions;
+    }
+
+    public static JoinGroupResponse parse(ByteBuffer buffer) {
+        return new JoinGroupResponse((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetRequest.java
new file mode 100644
index 0000000..069e06d
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetRequest.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ListOffsetRequest extends AbstractRequest {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id);
+    private static final String REPLICA_ID_KEY_NAME = "replica_id";
+    private static final String TOPICS_KEY_NAME = "topics";
+
+    // topic level field names
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partitions";
+
+    // partition level field names
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String TIMESTAMP_KEY_NAME = "timestamp";
+    private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets";
+
+    private final int replicaId;
+    private final Map<TopicPartition, PartitionData> offsetData;
+
+    public static final class PartitionData {
+        public final long timestamp;
+        public final int maxNumOffsets;
+
+        public PartitionData(long timestamp, int maxNumOffsets) {
+            this.timestamp = timestamp;
+            this.maxNumOffsets = maxNumOffsets;
+        }
+    }
+    
+    public ListOffsetRequest(Map<TopicPartition, PartitionData> offsetData) {
+        this(-1, offsetData);
+    }
+
+    public ListOffsetRequest(int replicaId, Map<TopicPartition, PartitionData> offsetData) {
+        super(new Struct(CURRENT_SCHEMA));
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
+
+        struct.set(REPLICA_ID_KEY_NAME, replicaId);
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData offsetPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp);
+                partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+        this.replicaId = replicaId;
+        this.offsetData = offsetData;
+    }
+
+    public ListOffsetRequest(Struct struct) {
+        super(struct);
+        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
+        offsetData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
+                int maxNumOffsets = partitionResponse.getInt(MAX_NUM_OFFSETS_KEY_NAME);
+                PartitionData partitionData = new PartitionData(timestamp, maxNumOffsets);
+                offsetData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        Map<TopicPartition, ListOffsetResponse.PartitionData> responseData = new HashMap<TopicPartition, ListOffsetResponse.PartitionData>();
+
+        for (Map.Entry<TopicPartition, PartitionData> entry: offsetData.entrySet()) {
+            ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(Errors.forException(e).code(), new ArrayList<Long>());
+            responseData.put(entry.getKey(), partitionResponse);
+        }
+
+        switch (versionId) {
+            case 0:
+                return new ListOffsetResponse(responseData);
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id)));
+        }
+    }
+
+    public int replicaId() {
+        return replicaId;
+    }
+
+    public Map<TopicPartition, PartitionData> offsetData() {
+        return offsetData;
+    }
+
+    public static ListOffsetRequest parse(ByteBuffer buffer, int versionId) {
+        return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer));
+    }
+
+    public static ListOffsetRequest parse(ByteBuffer buffer) {
+        return new ListOffsetRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetResponse.java
new file mode 100644
index 0000000..b831f61
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ListOffsetResponse.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ListOffsetResponse extends AbstractRequestResponse {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id);
+    private static final String RESPONSES_KEY_NAME = "responses";
+
+    // topic level field names
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partition_responses";
+
+    // partition level field names
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    /**
+     * Possible error code:
+     *
+     *  UNKNOWN_TOPIC_OR_PARTITION (3)
+     *  NOT_LEADER_FOR_PARTITION (6)
+     *  UNKNOWN (-1)
+     */
+
+    private static final String OFFSETS_KEY_NAME = "offsets";
+
+    private final Map<TopicPartition, PartitionData> responseData;
+
+    public static final class PartitionData {
+        public final short errorCode;
+        public final List<Long> offsets;
+
+        public PartitionData(short errorCode, List<Long> offsets) {
+            this.errorCode = errorCode;
+            this.offsets = offsets;
+        }
+    }
+
+    public ListOffsetResponse(Map<TopicPartition, PartitionData> responseData) {
+        super(new Struct(CURRENT_SCHEMA));
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData offsetPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.errorCode);
+                partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray());
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        this.responseData = responseData;
+    }
+
+    public ListOffsetResponse(Struct struct) {
+        super(struct);
+        responseData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+                Object[] offsets = partitionResponse.getArray(OFFSETS_KEY_NAME);
+                List<Long> offsetsList = new ArrayList<Long>();
+                for (Object offset: offsets)
+                    offsetsList.add((Long) offset);
+                PartitionData partitionData = new PartitionData(errorCode, offsetsList);
+                responseData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    public Map<TopicPartition, PartitionData> responseData() {
+        return responseData;
+    }
+
+    public static ListOffsetResponse parse(ByteBuffer buffer) {
+        return new ListOffsetResponse((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataRequest.java
new file mode 100644
index 0000000..2820fcd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataRequest.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.PartitionInfo;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.Cluster;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class MetadataRequest extends AbstractRequest {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id);
+    private static final String TOPICS_KEY_NAME = "topics";
+
+    private final List<String> topics;
+
+    public MetadataRequest(List<String> topics) {
+        super(new Struct(CURRENT_SCHEMA));
+        struct.set(TOPICS_KEY_NAME, topics.toArray());
+        this.topics = topics;
+    }
+
+    public MetadataRequest(Struct struct) {
+        super(struct);
+        Object[] topicArray = struct.getArray(TOPICS_KEY_NAME);
+        topics = new ArrayList<String>();
+        for (Object topicObj: topicArray) {
+            topics.add((String) topicObj);
+        }
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        Map<String, Errors> topicErrors = new HashMap<String, Errors>();
+        for (String topic : topics) {
+            topicErrors.put(topic, Errors.forException(e));
+        }
+
+        Cluster cluster = new Cluster(new ArrayList<Node>(), new ArrayList<PartitionInfo>());
+        switch (versionId) {
+            case 0:
+                return new MetadataResponse(cluster, topicErrors);
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id)));
+        }
+    }
+
+    public List<String> topics() {
+        return topics;
+    }
+
+    public static MetadataRequest parse(ByteBuffer buffer, int versionId) {
+        return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer));
+    }
+
+    public static MetadataRequest parse(ByteBuffer buffer) {
+        return new MetadataRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataResponse.java
new file mode 100644
index 0000000..83d7290
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/MetadataResponse.java
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.Cluster;
+import org.apache.flink.kafka_backport.common.PartitionInfo;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class MetadataResponse extends AbstractRequestResponse {
+
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id);
+    private static final String BROKERS_KEY_NAME = "brokers";
+    private static final String TOPIC_METATDATA_KEY_NAME = "topic_metadata";
+
+    // broker level field names
+    private static final String NODE_ID_KEY_NAME = "node_id";
+    private static final String HOST_KEY_NAME = "host";
+    private static final String PORT_KEY_NAME = "port";
+
+    // topic level field names
+    private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code";
+
+    /**
+     * Possible error code:
+     *
+     * TODO
+     */
+
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata";
+
+    // partition level field names
+    private static final String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code";
+
+    /**
+     * Possible error code:
+     *
+     * TODO
+     */
+
+    private static final String PARTITION_KEY_NAME = "partition_id";
+    private static final String LEADER_KEY_NAME = "leader";
+    private static final String REPLICAS_KEY_NAME = "replicas";
+    private static final String ISR_KEY_NAME = "isr";
+
+    private final Cluster cluster;
+    private final Map<String, Errors> errors;
+
+    /**
+     * Constructor for MetadataResponse where there are errors for some of the topics,
+     * error data take precedence over cluster information for particular topic
+     */
+    public MetadataResponse(Cluster cluster, Map<String, Errors> errors) {
+        super(new Struct(CURRENT_SCHEMA));
+
+        List<Struct> brokerArray = new ArrayList<Struct>();
+        for (Node node : cluster.nodes()) {
+            Struct broker = struct.instance(BROKERS_KEY_NAME);
+            broker.set(NODE_ID_KEY_NAME, node.id());
+            broker.set(HOST_KEY_NAME, node.host());
+            broker.set(PORT_KEY_NAME, node.port());
+            brokerArray.add(broker);
+        }
+        struct.set(BROKERS_KEY_NAME, brokerArray.toArray());
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (String topic : cluster.topics()) {
+            Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME);
+
+            topicData.set(TOPIC_KEY_NAME, topic);
+            if (errors.containsKey(topic)) {
+                topicData.set(TOPIC_ERROR_CODE_KEY_NAME, errors.get(topic).code());
+            } else {
+                topicData.set(TOPIC_ERROR_CODE_KEY_NAME, Errors.NONE.code());
+                List<Struct> partitionArray = new ArrayList<Struct>();
+                for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) {
+                    Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME);
+                    partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, Errors.NONE.code());
+                    partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition());
+                    partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id());
+                    ArrayList<Integer> replicas = new ArrayList<Integer>();
+                    for (Node node : fetchPartitionData.replicas())
+                        replicas.add(node.id());
+                    partitionData.set(REPLICAS_KEY_NAME, replicas.toArray());
+                    ArrayList<Integer> isr = new ArrayList<Integer>();
+                    for (Node node : fetchPartitionData.inSyncReplicas())
+                        isr.add(node.id());
+                    partitionData.set(ISR_KEY_NAME, isr.toArray());
+                    partitionArray.add(partitionData);
+                }
+                topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray());
+            }
+
+            topicArray.add(topicData);
+        }
+        struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray());
+
+        this.cluster = cluster;
+        this.errors = new HashMap<String, Errors>();
+    }
+
+    public MetadataResponse(Struct struct) {
+        super(struct);
+        Map<String, Errors> errors = new HashMap<String, Errors>();
+        Map<Integer, Node> brokers = new HashMap<Integer, Node>();
+        Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME);
+        for (int i = 0; i < brokerStructs.length; i++) {
+            Struct broker = (Struct) brokerStructs[i];
+            int nodeId = broker.getInt(NODE_ID_KEY_NAME);
+            String host = broker.getString(HOST_KEY_NAME);
+            int port = broker.getInt(PORT_KEY_NAME);
+            brokers.put(nodeId, new Node(nodeId, host, port));
+        }
+        List<PartitionInfo> partitions = new ArrayList<PartitionInfo>();
+        Object[] topicInfos = (Object[]) struct.get(TOPIC_METATDATA_KEY_NAME);
+        for (int i = 0; i < topicInfos.length; i++) {
+            Struct topicInfo = (Struct) topicInfos[i];
+            short topicError = topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME);
+            String topic = topicInfo.getString(TOPIC_KEY_NAME);
+            if (topicError == Errors.NONE.code()) {
+                Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME);
+                for (int j = 0; j < partitionInfos.length; j++) {
+                    Struct partitionInfo = (Struct) partitionInfos[j];
+                    int partition = partitionInfo.getInt(PARTITION_KEY_NAME);
+                    int leader = partitionInfo.getInt(LEADER_KEY_NAME);
+                    Node leaderNode = leader == -1 ? null : brokers.get(leader);
+                    Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME);
+                    Node[] replicaNodes = new Node[replicas.length];
+                    for (int k = 0; k < replicas.length; k++)
+                        replicaNodes[k] = brokers.get(replicas[k]);
+                    Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME);
+                    Node[] isrNodes = new Node[isr.length];
+                    for (int k = 0; k < isr.length; k++)
+                        isrNodes[k] = brokers.get(isr[k]);
+                    partitions.add(new PartitionInfo(topic, partition, leaderNode, replicaNodes, isrNodes));
+                }
+            } else {
+                errors.put(topic, Errors.forCode(topicError));
+            }
+        }
+        this.errors = errors;
+        this.cluster = new Cluster(brokers.values(), partitions);
+    }
+
+    public Map<String, Errors> errors() {
+        return this.errors;
+    }
+
+    public Cluster cluster() {
+        return this.cluster;
+    }
+
+    public static MetadataResponse parse(ByteBuffer buffer) {
+        return new MetadataResponse((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitRequest.java
new file mode 100644
index 0000000..b33d2c1
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitRequest.java
@@ -0,0 +1,275 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * This wrapper supports both v0 and v1 of OffsetCommitRequest.
+ */
+public class OffsetCommitRequest extends AbstractRequest {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String GENERATION_ID_KEY_NAME = "group_generation_id";
+    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
+    private static final String TOPICS_KEY_NAME = "topics";
+    private static final String RETENTION_TIME_KEY_NAME = "retention_time";
+
+    // topic level field names
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partitions";
+
+    // partition level field names
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
+    private static final String METADATA_KEY_NAME = "metadata";
+
+    @Deprecated
+    private static final String TIMESTAMP_KEY_NAME = "timestamp";         // for v0, v1
+
+    // default values for the current version
+    public static final int DEFAULT_GENERATION_ID = -1;
+    public static final String DEFAULT_CONSUMER_ID = "";
+    public static final long DEFAULT_RETENTION_TIME = -1L;
+
+    // default values for old versions,
+    // will be removed after these versions are deprecated
+    @Deprecated
+    public static final long DEFAULT_TIMESTAMP = -1L;            // for V0, V1
+
+    private final String groupId;
+    private final String consumerId;
+    private final int generationId;
+    private final long retentionTime;
+    private final Map<TopicPartition, PartitionData> offsetData;
+
+    public static final class PartitionData {
+        @Deprecated
+        public final long timestamp;                // for V1
+
+        public final long offset;
+        public final String metadata;
+
+        @Deprecated
+        public PartitionData(long offset, long timestamp, String metadata) {
+            this.offset = offset;
+            this.timestamp = timestamp;
+            this.metadata = metadata;
+        }
+
+        public PartitionData(long offset, String metadata) {
+            this(offset, DEFAULT_TIMESTAMP, metadata);
+        }
+    }
+
+    /**
+     * Constructor for version 0.
+     * @param groupId
+     * @param offsetData
+     */
+    @Deprecated
+    public OffsetCommitRequest(String groupId, Map<TopicPartition, PartitionData> offsetData) {
+        super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0)));
+
+        initCommonFields(groupId, offsetData);
+        this.groupId = groupId;
+        this.generationId = DEFAULT_GENERATION_ID;
+        this.consumerId = DEFAULT_CONSUMER_ID;
+        this.retentionTime = DEFAULT_RETENTION_TIME;
+        this.offsetData = offsetData;
+    }
+
+    /**
+     * Constructor for version 1.
+     * @param groupId
+     * @param generationId
+     * @param consumerId
+     * @param offsetData
+     */
+    @Deprecated
+    public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map<TopicPartition, PartitionData> offsetData) {
+        super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1)));
+
+        initCommonFields(groupId, offsetData);
+        struct.set(GENERATION_ID_KEY_NAME, generationId);
+        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        this.groupId = groupId;
+        this.generationId = generationId;
+        this.consumerId = consumerId;
+        this.retentionTime = DEFAULT_RETENTION_TIME;
+        this.offsetData = offsetData;
+    }
+
+    /**
+     * Constructor for version 2.
+     * @param groupId
+     * @param generationId
+     * @param consumerId
+     * @param retentionTime
+     * @param offsetData
+     */
+    public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map<TopicPartition, PartitionData> offsetData) {
+        super(new Struct(CURRENT_SCHEMA));
+
+        initCommonFields(groupId, offsetData);
+        struct.set(GENERATION_ID_KEY_NAME, generationId);
+        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        struct.set(RETENTION_TIME_KEY_NAME, retentionTime);
+        this.groupId = groupId;
+        this.generationId = generationId;
+        this.consumerId = consumerId;
+        this.retentionTime = retentionTime;
+        this.offsetData = offsetData;
+    }
+
+    private void initCommonFields(String groupId, Map<TopicPartition, PartitionData> offsetData) {
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
+
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        List<Struct> topicArray = new ArrayList<Struct>();
+
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData fetchPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
+                // Only for v1
+                if (partitionData.hasField(TIMESTAMP_KEY_NAME))
+                    partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp);
+                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+    }
+
+    public OffsetCommitRequest(Struct struct) {
+        super(struct);
+
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+        // This field only exists in v1.
+        if (struct.hasField(GENERATION_ID_KEY_NAME))
+            generationId = struct.getInt(GENERATION_ID_KEY_NAME);
+        else
+            generationId = DEFAULT_GENERATION_ID;
+
+        // This field only exists in v1.
+        if (struct.hasField(CONSUMER_ID_KEY_NAME))
+            consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+        else
+            consumerId = DEFAULT_CONSUMER_ID;
+
+        // This field only exists in v2
+        if (struct.hasField(RETENTION_TIME_KEY_NAME))
+            retentionTime = struct.getLong(RETENTION_TIME_KEY_NAME);
+        else
+            retentionTime = DEFAULT_RETENTION_TIME;
+
+        offsetData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicDataObj : struct.getArray(TOPICS_KEY_NAME)) {
+            Struct topicData = (Struct) topicDataObj;
+            String topic = topicData.getString(TOPIC_KEY_NAME);
+            for (Object partitionDataObj : topicData.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionDataStruct = (Struct) partitionDataObj;
+                int partition = partitionDataStruct.getInt(PARTITION_KEY_NAME);
+                long offset = partitionDataStruct.getLong(COMMIT_OFFSET_KEY_NAME);
+                String metadata = partitionDataStruct.getString(METADATA_KEY_NAME);
+                PartitionData partitionOffset;
+                // This field only exists in v1
+                if (partitionDataStruct.hasField(TIMESTAMP_KEY_NAME)) {
+                    long timestamp = partitionDataStruct.getLong(TIMESTAMP_KEY_NAME);
+                    partitionOffset = new PartitionData(offset, timestamp, metadata);
+                } else {
+                    partitionOffset = new PartitionData(offset, metadata);
+                }
+                offsetData.put(new TopicPartition(topic, partition), partitionOffset);
+            }
+        }
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        Map<TopicPartition, Short> responseData = new HashMap<TopicPartition, Short>();
+        for (Map.Entry<TopicPartition, PartitionData> entry: offsetData.entrySet()) {
+            responseData.put(entry.getKey(), Errors.forException(e).code());
+        }
+
+        switch (versionId) {
+            // OffsetCommitResponseV0 == OffsetCommitResponseV1 == OffsetCommitResponseV2
+            case 0:
+            case 1:
+            case 2:
+                return new OffsetCommitResponse(responseData);
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_COMMIT.id)));
+        }
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public int generationId() {
+        return generationId;
+    }
+
+    public String consumerId() {
+        return consumerId;
+    }
+
+    public long retentionTime() {
+        return retentionTime;
+    }
+
+    public Map<TopicPartition, PartitionData> offsetData() {
+        return offsetData;
+    }
+
+    public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) {
+        Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId);
+        return new OffsetCommitRequest((Struct) schema.read(buffer));
+    }
+
+    public static OffsetCommitRequest parse(ByteBuffer buffer) {
+        return new OffsetCommitRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitResponse.java
new file mode 100644
index 0000000..5f14b63
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/OffsetCommitResponse.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.TopicPartition;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class OffsetCommitResponse extends AbstractRequestResponse {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id);
+    private static final String RESPONSES_KEY_NAME = "responses";
+
+    // topic level fields
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partition_responses";
+
+    // partition level fields
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    /**
+     * Possible error code:
+     *
+     * OFFSET_METADATA_TOO_LARGE (12)
+     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
+     * NOT_COORDINATOR_FOR_CONSUMER (16)
+     * ILLEGAL_GENERATION (22)
+     * UNKNOWN_CONSUMER_ID (25)
+     * COMMITTING_PARTITIONS_NOT_ASSIGNED (27)
+     * INVALID_COMMIT_OFFSET_SIZE (28)
+     */
+
+    private final Map<TopicPartition, Short> responseData;
+
+    public OffsetCommitResponse(Map<TopicPartition, Short> responseData) {
+        super(new Struct(CURRENT_SCHEMA));
+
+        Map<String, Map<Integer, Short>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, Short>> entries: topicsData.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, Short> partitionEntry : entries.getValue().entrySet()) {
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(ERROR_CODE_KEY_NAME, partitionEntry.getValue());
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        this.responseData = responseData;
+    }
+
+    public OffsetCommitResponse(Struct struct) {
+        super(struct);
+        responseData = new HashMap<TopicPartition, Short>();
+        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+                responseData.put(new TopicPartition(topic, partition), errorCode);
+            }
+        }
+    }
+
+    public Map<TopicPartition, Short> responseData() {
+        return responseData;
+    }
+
+    public static OffsetCommitResponse parse(ByteBuffer buffer) {
+        return new OffsetCommitResponse((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}


[15/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Compressor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Compressor.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Compressor.java
new file mode 100644
index 0000000..b1b537c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Compressor.java
@@ -0,0 +1,279 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.record;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class Compressor {
+
+    static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f;
+    static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f;
+    static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024;
+
+    private static final float[] TYPE_TO_RATE;
+
+    static {
+        int maxTypeId = -1;
+        for (CompressionType type : CompressionType.values())
+            maxTypeId = Math.max(maxTypeId, type.id);
+        TYPE_TO_RATE = new float[maxTypeId + 1];
+        for (CompressionType type : CompressionType.values()) {
+            TYPE_TO_RATE[type.id] = type.rate;
+        }
+    }
+
+    private final CompressionType type;
+    private final DataOutputStream appendStream;
+    private final ByteBufferOutputStream bufferStream;
+    private final int initPos;
+
+    public long writtenUncompressed;
+    public long numRecords;
+
+    public Compressor(ByteBuffer buffer, CompressionType type, int blockSize) {
+        this.type = type;
+        this.initPos = buffer.position();
+
+        this.numRecords = 0;
+        this.writtenUncompressed = 0;
+
+        if (type != CompressionType.NONE) {
+            // for compressed records, leave space for the header and the shallow message metadata
+            // and move the starting position to the value payload offset
+            buffer.position(initPos + Records.LOG_OVERHEAD + Record.RECORD_OVERHEAD);
+        }
+
+        // create the stream
+        bufferStream = new ByteBufferOutputStream(buffer);
+        appendStream = wrapForOutput(bufferStream, type, blockSize);
+    }
+
+    public Compressor(ByteBuffer buffer, CompressionType type) {
+        this(buffer, type, COMPRESSION_DEFAULT_BUFFER_SIZE);
+    }
+
+    public ByteBuffer buffer() {
+        return bufferStream.buffer();
+    }
+    
+    public double compressionRate() {
+        ByteBuffer buffer = bufferStream.buffer();
+        if (this.writtenUncompressed == 0)
+            return 1.0;
+        else
+            return (double) buffer.position() / this.writtenUncompressed;
+    }
+
+    public void close() {
+        try {
+            appendStream.close();
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+
+        if (type != CompressionType.NONE) {
+            ByteBuffer buffer = bufferStream.buffer();
+            int pos = buffer.position();
+            // write the header, for the end offset write as number of records - 1
+            buffer.position(initPos);
+            buffer.putLong(numRecords - 1);
+            buffer.putInt(pos - initPos - Records.LOG_OVERHEAD);
+            // write the shallow message (the crc and value size are not correct yet)
+            Record.write(buffer, null, null, type, 0, -1);
+            // compute the fill the value size
+            int valueSize = pos - initPos - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD;
+            buffer.putInt(initPos + Records.LOG_OVERHEAD + Record.KEY_OFFSET, valueSize);
+            // compute and fill the crc at the beginning of the message
+            long crc = Record.computeChecksum(buffer,
+                    initPos + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET,
+                    pos - initPos - Records.LOG_OVERHEAD - Record.MAGIC_OFFSET);
+            Utils.writeUnsignedInt(buffer, initPos + Records.LOG_OVERHEAD + Record.CRC_OFFSET, crc);
+            // reset the position
+            buffer.position(pos);
+
+            // update the compression ratio
+            float compressionRate = (float) buffer.position() / this.writtenUncompressed;
+            TYPE_TO_RATE[type.id] = TYPE_TO_RATE[type.id] * COMPRESSION_RATE_DAMPING_FACTOR +
+                compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR);
+        }
+    }
+
+    // Note that for all the write operations below, IO exceptions should
+    // never be thrown since the underlying ByteBufferOutputStream does not throw IOException;
+    // therefore upon encountering this issue we just close the append stream.
+
+    public void putLong(final long value) {
+        try {
+            appendStream.writeLong(value);
+        } catch (IOException e) {
+            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+        }
+    }
+
+    public void putInt(final int value) {
+        try {
+            appendStream.writeInt(value);
+        } catch (IOException e) {
+            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+        }
+    }
+
+    public void put(final ByteBuffer buffer) {
+        try {
+            appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit());
+        } catch (IOException e) {
+            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+        }
+    }
+
+    public void putByte(final byte value) {
+        try {
+            appendStream.write(value);
+        } catch (IOException e) {
+            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+        }
+    }
+
+    public void put(final byte[] bytes, final int offset, final int len) {
+        try {
+            appendStream.write(bytes, offset, len);
+        } catch (IOException e) {
+            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+        }
+    }
+
+    public void putRecord(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
+        // put a record as un-compressed into the underlying stream
+        long crc = Record.computeChecksum(key, value, type, valueOffset, valueSize);
+        byte attributes = Record.computeAttributes(type);
+        putRecord(crc, attributes, key, value, valueOffset, valueSize);
+    }
+
+    public void putRecord(byte[] key, byte[] value) {
+        putRecord(key, value, CompressionType.NONE, 0, -1);
+    }
+
+    private void putRecord(final long crc, final byte attributes, final byte[] key, final byte[] value, final int valueOffset, final int valueSize) {
+        Record.write(this, crc, attributes, key, value, valueOffset, valueSize);
+    }
+
+    public void recordWritten(int size) {
+        numRecords += 1;
+        writtenUncompressed += size;
+    }
+
+    public long estimatedBytesWritten() {
+        if (type == CompressionType.NONE) {
+            return bufferStream.buffer().position();
+        } else {
+            // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
+            return (long) (writtenUncompressed * TYPE_TO_RATE[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR);
+        }
+    }
+
+    // the following two functions also need to be public since they are used in MemoryRecords.iteration
+
+    static public DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, int bufferSize) {
+        try {
+            switch (type) {
+                case NONE:
+                    return new DataOutputStream(buffer);
+                case GZIP:
+                    return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize));
+                case SNAPPY:
+                    // dynamically load the snappy class to avoid runtime dependency
+                    // on snappy if we are not using it
+                    try {
+                        Class<?> outputStreamClass = Class.forName("org.xerial.snappy.SnappyOutputStream");
+                        OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class, Integer.TYPE)
+                            .newInstance(buffer, bufferSize);
+                        return new DataOutputStream(stream);
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                case LZ4:
+                    try {
+                        Class<?> outputStreamClass = Class.forName("org.apache.KafkaLZ4BlockOutputStream");
+                        OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class)
+                            .newInstance(buffer);
+                        return new DataOutputStream(stream);
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                default:
+                    throw new IllegalArgumentException("Unknown compression type: " + type);
+            }
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    static public DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type) {
+        try {
+            switch (type) {
+                case NONE:
+                    return new DataInputStream(buffer);
+                case GZIP:
+                    return new DataInputStream(new GZIPInputStream(buffer));
+                case SNAPPY:
+                    // dynamically load the snappy class to avoid runtime dependency
+                    // on snappy if we are not using it
+                    try {
+                        Class<?> inputStreamClass = Class.forName("org.xerial.snappy.SnappyInputStream");
+                        InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class)
+                            .newInstance(buffer);
+                        return new DataInputStream(stream);
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                case LZ4:
+                    // dynamically load LZ4 class to avoid runtime dependency
+                    try {
+                        Class<?> inputStreamClass = Class.forName("org.apache.KafkaLZ4BlockInputStream");
+                        InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class)
+                            .newInstance(buffer);
+                        return new DataInputStream(stream);
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                default:
+                    throw new IllegalArgumentException("Unknown compression type: " + type);
+            }
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/InvalidRecordException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/InvalidRecordException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/InvalidRecordException.java
new file mode 100644
index 0000000..38c26ed
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/InvalidRecordException.java
@@ -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.
+ */
+package org.apache.flink.kafka_backport.common.record;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class InvalidRecordException extends RuntimeException {
+
+    private static final long serialVersionUID = 1;
+
+    public InvalidRecordException(String s) {
+        super(s);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockInputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockInputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockInputStream.java
new file mode 100644
index 0000000..1049787
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockInputStream.java
@@ -0,0 +1,236 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * 
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kafka_backport.common.record;
+
+import net.jpountz.lz4.LZ4Exception;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.lz4.LZ4SafeDecompressor;
+import net.jpountz.xxhash.XXHash32;
+import net.jpountz.xxhash.XXHashFactory;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A partial implementation of the v1.4.1 LZ4 Frame format.
+ * 
+ * @see <a href="https://docs.google.com/document/d/1Tdxmn5_2e5p1y4PtXkatLndWVb0R8QARJFe6JI4Keuo/edit">LZ4 Framing
+ *      Format Spec</a>
+ */
+public final class KafkaLZ4BlockInputStream extends FilterInputStream {
+
+    public static final String PREMATURE_EOS = "Stream ended prematurely";
+    public static final String NOT_SUPPORTED = "Stream unsupported";
+    public static final String BLOCK_HASH_MISMATCH = "Block checksum mismatch";
+    public static final String DESCRIPTOR_HASH_MISMATCH = "Stream frame descriptor corrupted";
+
+    private final LZ4SafeDecompressor decompressor;
+    private final XXHash32 checksum;
+    private final byte[] buffer;
+    private final byte[] compressedBuffer;
+    private final int maxBlockSize;
+    private KafkaLZ4BlockOutputStream.FLG flg;
+    private KafkaLZ4BlockOutputStream.BD bd;
+    private int bufferOffset;
+    private int bufferSize;
+    private boolean finished;
+
+    /**
+     * Create a new {@link InputStream} that will decompress data using the LZ4 algorithm.
+     * 
+     * @param in The stream to decompress
+     * @throws IOException
+     */
+    public KafkaLZ4BlockInputStream(InputStream in) throws IOException {
+        super(in);
+        decompressor = LZ4Factory.fastestInstance().safeDecompressor();
+        checksum = XXHashFactory.fastestInstance().hash32();
+        readHeader();
+        maxBlockSize = bd.getBlockMaximumSize();
+        buffer = new byte[maxBlockSize];
+        compressedBuffer = new byte[maxBlockSize];
+        bufferOffset = 0;
+        bufferSize = 0;
+        finished = false;
+    }
+
+    /**
+     * Reads the magic number and frame descriptor from the underlying {@link InputStream}.
+     * 
+     * @throws IOException
+     */
+    private void readHeader() throws IOException {
+        byte[] header = new byte[KafkaLZ4BlockOutputStream.LZ4_MAX_HEADER_LENGTH];
+
+        // read first 6 bytes into buffer to check magic and FLG/BD descriptor flags
+        bufferOffset = 6;
+        if (in.read(header, 0, bufferOffset) != bufferOffset) {
+            throw new IOException(PREMATURE_EOS);
+        }
+
+        if (KafkaLZ4BlockOutputStream.MAGIC != Utils.readUnsignedIntLE(header, bufferOffset - 6)) {
+            throw new IOException(NOT_SUPPORTED);
+        }
+        flg = KafkaLZ4BlockOutputStream.FLG.fromByte(header[bufferOffset - 2]);
+        bd = KafkaLZ4BlockOutputStream.BD.fromByte(header[bufferOffset - 1]);
+        // TODO read uncompressed content size, update flg.validate()
+        // TODO read dictionary id, update flg.validate()
+
+        // check stream descriptor hash
+        byte hash = (byte) ((checksum.hash(header, 0, bufferOffset, 0) >> 8) & 0xFF);
+        header[bufferOffset++] = (byte) in.read();
+        if (hash != header[bufferOffset - 1]) {
+            throw new IOException(DESCRIPTOR_HASH_MISMATCH);
+        }
+    }
+
+    /**
+     * Decompresses (if necessary) buffered data, optionally computes and validates a XXHash32 checksum, and writes the
+     * result to a buffer.
+     * 
+     * @throws IOException
+     */
+    private void readBlock() throws IOException {
+        int blockSize = Utils.readUnsignedIntLE(in);
+
+        // Check for EndMark
+        if (blockSize == 0) {
+            finished = true;
+            // TODO implement content checksum, update flg.validate()
+            return;
+        } else if (blockSize > maxBlockSize) {
+            throw new IOException(String.format("Block size %s exceeded max: %s", blockSize, maxBlockSize));
+        }
+
+        boolean compressed = (blockSize & KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK) == 0;
+        byte[] bufferToRead;
+        if (compressed) {
+            bufferToRead = compressedBuffer;
+        } else {
+            blockSize &= ~KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK;
+            bufferToRead = buffer;
+            bufferSize = blockSize;
+        }
+
+        if (in.read(bufferToRead, 0, blockSize) != blockSize) {
+            throw new IOException(PREMATURE_EOS);
+        }
+
+        // verify checksum
+        if (flg.isBlockChecksumSet() && Utils.readUnsignedIntLE(in) != checksum.hash(bufferToRead, 0, blockSize, 0)) {
+            throw new IOException(BLOCK_HASH_MISMATCH);
+        }
+
+        if (compressed) {
+            try {
+                bufferSize = decompressor.decompress(compressedBuffer, 0, blockSize, buffer, 0, maxBlockSize);
+            } catch (LZ4Exception e) {
+                throw new IOException(e);
+            }
+        }
+
+        bufferOffset = 0;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (finished) {
+            return -1;
+        }
+        if (available() == 0) {
+            readBlock();
+        }
+        if (finished) {
+            return -1;
+        }
+        int value = buffer[bufferOffset++] & 0xFF;
+
+        return value;
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        net.jpountz.util.Utils.checkRange(b, off, len);
+        if (finished) {
+            return -1;
+        }
+        if (available() == 0) {
+            readBlock();
+        }
+        if (finished) {
+            return -1;
+        }
+        len = Math.min(len, available());
+        System.arraycopy(buffer, bufferOffset, b, off, len);
+        bufferOffset += len;
+        return len;
+    }
+
+    @Override
+    public long skip(long n) throws IOException {
+        if (finished) {
+            return 0;
+        }
+        if (available() == 0) {
+            readBlock();
+        }
+        if (finished) {
+            return 0;
+        }
+        n = Math.min(n, available());
+        bufferOffset += n;
+        return n;
+    }
+
+    @Override
+    public int available() throws IOException {
+        return bufferSize - bufferOffset;
+    }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
+
+    @Override
+    public synchronized void mark(int readlimit) {
+        throw new RuntimeException("mark not supported");
+    }
+
+    @Override
+    public synchronized void reset() throws IOException {
+        throw new RuntimeException("reset not supported");
+    }
+
+    @Override
+    public boolean markSupported() {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockOutputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockOutputStream.java
new file mode 100644
index 0000000..3007c42
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/KafkaLZ4BlockOutputStream.java
@@ -0,0 +1,400 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * 
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kafka_backport.common.record;
+
+import net.jpountz.lz4.LZ4Compressor;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.xxhash.XXHash32;
+import net.jpountz.xxhash.XXHashFactory;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A partial implementation of the v1.4.1 LZ4 Frame format.
+ * 
+ * @see <a href="https://docs.google.com/document/d/1Tdxmn5_2e5p1y4PtXkatLndWVb0R8QARJFe6JI4Keuo/edit">LZ4 Framing
+ *      Format Spec</a>
+ */
+public final class KafkaLZ4BlockOutputStream extends FilterOutputStream {
+
+    public static final int MAGIC = 0x184D2204;
+    public static final int LZ4_MAX_HEADER_LENGTH = 19;
+    public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000;
+
+    public static final String CLOSED_STREAM = "The stream is already closed";
+
+    public static final int BLOCKSIZE_64KB = 4;
+    public static final int BLOCKSIZE_256KB = 5;
+    public static final int BLOCKSIZE_1MB = 6;
+    public static final int BLOCKSIZE_4MB = 7;
+
+    private final LZ4Compressor compressor;
+    private final XXHash32 checksum;
+    private final FLG flg;
+    private final BD bd;
+    private final byte[] buffer;
+    private final byte[] compressedBuffer;
+    private final int maxBlockSize;
+    private int bufferOffset;
+    private boolean finished;
+
+    /**
+     * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
+     * 
+     * @param out The output stream to compress
+     * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other
+     *            values will generate an exception
+     * @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for
+     *            every block of data
+     * @throws IOException
+     */
+    public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum) throws IOException {
+        super(out);
+        compressor = LZ4Factory.fastestInstance().fastCompressor();
+        checksum = XXHashFactory.fastestInstance().hash32();
+        bd = new BD(blockSize);
+        flg = new FLG(blockChecksum);
+        bufferOffset = 0;
+        maxBlockSize = bd.getBlockMaximumSize();
+        buffer = new byte[maxBlockSize];
+        compressedBuffer = new byte[compressor.maxCompressedLength(maxBlockSize)];
+        finished = false;
+        writeHeader();
+    }
+
+    /**
+     * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
+     * 
+     * @param out The stream to compress
+     * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other
+     *            values will generate an exception
+     * @throws IOException
+     */
+    public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize) throws IOException {
+        this(out, blockSize, false);
+    }
+
+    /**
+     * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
+     * 
+     * @param out The output stream to compress
+     * @throws IOException
+     */
+    public KafkaLZ4BlockOutputStream(OutputStream out) throws IOException {
+        this(out, BLOCKSIZE_64KB);
+    }
+
+    /**
+     * Writes the magic number and frame descriptor to the underlying {@link OutputStream}.
+     * 
+     * @throws IOException
+     */
+    private void writeHeader() throws IOException {
+        Utils.writeUnsignedIntLE(buffer, 0, MAGIC);
+        bufferOffset = 4;
+        buffer[bufferOffset++] = flg.toByte();
+        buffer[bufferOffset++] = bd.toByte();
+        // TODO write uncompressed content size, update flg.validate()
+        // TODO write dictionary id, update flg.validate()
+        // compute checksum on all descriptor fields
+        int hash = (checksum.hash(buffer, 0, bufferOffset, 0) >> 8) & 0xFF;
+        buffer[bufferOffset++] = (byte) hash;
+        // write out frame descriptor
+        out.write(buffer, 0, bufferOffset);
+        bufferOffset = 0;
+    }
+
+    /**
+     * Compresses buffered data, optionally computes an XXHash32 checksum, and writes the result to the underlying
+     * {@link OutputStream}.
+     * 
+     * @throws IOException
+     */
+    private void writeBlock() throws IOException {
+        if (bufferOffset == 0) {
+            return;
+        }
+
+        int compressedLength = compressor.compress(buffer, 0, bufferOffset, compressedBuffer, 0);
+        byte[] bufferToWrite = compressedBuffer;
+        int compressMethod = 0;
+
+        // Store block uncompressed if compressed length is greater (incompressible)
+        if (compressedLength >= bufferOffset) {
+            bufferToWrite = buffer;
+            compressedLength = bufferOffset;
+            compressMethod = LZ4_FRAME_INCOMPRESSIBLE_MASK;
+        }
+
+        // Write content
+        Utils.writeUnsignedIntLE(out, compressedLength | compressMethod);
+        out.write(bufferToWrite, 0, compressedLength);
+
+        // Calculate and write block checksum
+        if (flg.isBlockChecksumSet()) {
+            int hash = checksum.hash(bufferToWrite, 0, compressedLength, 0);
+            Utils.writeUnsignedIntLE(out, hash);
+        }
+        bufferOffset = 0;
+    }
+
+    /**
+     * Similar to the {@link #writeBlock()} method. Writes a 0-length block (without block checksum) to signal the end
+     * of the block stream.
+     * 
+     * @throws IOException
+     */
+    private void writeEndMark() throws IOException {
+        Utils.writeUnsignedIntLE(out, 0);
+        // TODO implement content checksum, update flg.validate()
+        finished = true;
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        ensureNotFinished();
+        if (bufferOffset == maxBlockSize) {
+            writeBlock();
+        }
+        buffer[bufferOffset++] = (byte) b;
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+        net.jpountz.util.Utils.checkRange(b, off, len);
+        ensureNotFinished();
+
+        int bufferRemainingLength = maxBlockSize - bufferOffset;
+        // while b will fill the buffer
+        while (len > bufferRemainingLength) {
+            // fill remaining space in buffer
+            System.arraycopy(b, off, buffer, bufferOffset, bufferRemainingLength);
+            bufferOffset = maxBlockSize;
+            writeBlock();
+            // compute new offset and length
+            off += bufferRemainingLength;
+            len -= bufferRemainingLength;
+            bufferRemainingLength = maxBlockSize;
+        }
+
+        System.arraycopy(b, off, buffer, bufferOffset, len);
+        bufferOffset += len;
+    }
+
+    @Override
+    public void flush() throws IOException {
+        if (!finished) {
+            writeBlock();
+        }
+        if (out != null) {
+            out.flush();
+        }
+    }
+
+    /**
+     * A simple state check to ensure the stream is still open.
+     */
+    private void ensureNotFinished() {
+        if (finished) {
+            throw new IllegalStateException(CLOSED_STREAM);
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (!finished) {
+            writeEndMark();
+            flush();
+            finished = true;
+        }
+        if (out != null) {
+            out.close();
+            out = null;
+        }
+    }
+
+    public static class FLG {
+
+        private static final int VERSION = 1;
+
+        private final int presetDictionary;
+        private final int reserved1;
+        private final int contentChecksum;
+        private final int contentSize;
+        private final int blockChecksum;
+        private final int blockIndependence;
+        private final int version;
+
+        public FLG() {
+            this(false);
+        }
+
+        public FLG(boolean blockChecksum) {
+            this(0, 0, 0, 0, blockChecksum ? 1 : 0, 1, VERSION);
+        }
+
+        private FLG(int presetDictionary,
+                    int reserved1,
+                    int contentChecksum,
+                    int contentSize,
+                    int blockChecksum,
+                    int blockIndependence,
+                    int version) {
+            this.presetDictionary = presetDictionary;
+            this.reserved1 = reserved1;
+            this.contentChecksum = contentChecksum;
+            this.contentSize = contentSize;
+            this.blockChecksum = blockChecksum;
+            this.blockIndependence = blockIndependence;
+            this.version = version;
+            validate();
+        }
+
+        public static FLG fromByte(byte flg) {
+            int presetDictionary = (flg >>> 0) & 1;
+            int reserved1 = (flg >>> 1) & 1;
+            int contentChecksum = (flg >>> 2) & 1;
+            int contentSize = (flg >>> 3) & 1;
+            int blockChecksum = (flg >>> 4) & 1;
+            int blockIndependence = (flg >>> 5) & 1;
+            int version = (flg >>> 6) & 3;
+
+            return new FLG(presetDictionary,
+                           reserved1,
+                           contentChecksum,
+                           contentSize,
+                           blockChecksum,
+                           blockIndependence,
+                           version);
+        }
+
+        public byte toByte() {
+            return (byte) (((presetDictionary & 1) << 0) | ((reserved1 & 1) << 1) | ((contentChecksum & 1) << 2)
+                    | ((contentSize & 1) << 3) | ((blockChecksum & 1) << 4) | ((blockIndependence & 1) << 5) | ((version & 3) << 6));
+        }
+
+        private void validate() {
+            if (presetDictionary != 0) {
+                throw new RuntimeException("Preset dictionary is unsupported");
+            }
+            if (reserved1 != 0) {
+                throw new RuntimeException("Reserved1 field must be 0");
+            }
+            if (contentChecksum != 0) {
+                throw new RuntimeException("Content checksum is unsupported");
+            }
+            if (contentSize != 0) {
+                throw new RuntimeException("Content size is unsupported");
+            }
+            if (blockIndependence != 1) {
+                throw new RuntimeException("Dependent block stream is unsupported");
+            }
+            if (version != VERSION) {
+                throw new RuntimeException(String.format("Version %d is unsupported", version));
+            }
+        }
+
+        public boolean isPresetDictionarySet() {
+            return presetDictionary == 1;
+        }
+
+        public boolean isContentChecksumSet() {
+            return contentChecksum == 1;
+        }
+
+        public boolean isContentSizeSet() {
+            return contentSize == 1;
+        }
+
+        public boolean isBlockChecksumSet() {
+            return blockChecksum == 1;
+        }
+
+        public boolean isBlockIndependenceSet() {
+            return blockIndependence == 1;
+        }
+
+        public int getVersion() {
+            return version;
+        }
+    }
+
+    public static class BD {
+
+        private final int reserved2;
+        private final int blockSizeValue;
+        private final int reserved3;
+
+        public BD() {
+            this(0, BLOCKSIZE_64KB, 0);
+        }
+
+        public BD(int blockSizeValue) {
+            this(0, blockSizeValue, 0);
+        }
+
+        private BD(int reserved2, int blockSizeValue, int reserved3) {
+            this.reserved2 = reserved2;
+            this.blockSizeValue = blockSizeValue;
+            this.reserved3 = reserved3;
+            validate();
+        }
+
+        public static BD fromByte(byte bd) {
+            int reserved2 = (bd >>> 0) & 15;
+            int blockMaximumSize = (bd >>> 4) & 7;
+            int reserved3 = (bd >>> 7) & 1;
+
+            return new BD(reserved2, blockMaximumSize, reserved3);
+        }
+
+        private void validate() {
+            if (reserved2 != 0) {
+                throw new RuntimeException("Reserved2 field must be 0");
+            }
+            if (blockSizeValue < 4 || blockSizeValue > 7) {
+                throw new RuntimeException("Block size value must be between 4 and 7");
+            }
+            if (reserved3 != 0) {
+                throw new RuntimeException("Reserved3 field must be 0");
+            }
+        }
+
+        // 2^(2n+8)
+        public int getBlockMaximumSize() {
+            return 1 << ((2 * blockSizeValue) + 8);
+        }
+
+        public byte toByte() {
+            return (byte) (((reserved2 & 15) << 0) | ((blockSizeValue & 7) << 4) | ((reserved3 & 1) << 7));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/LogEntry.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/LogEntry.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/LogEntry.java
new file mode 100644
index 0000000..b5d7b2d
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/LogEntry.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.kafka_backport.common.record;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * An offset and record pair
+ */
+public final class LogEntry {
+
+    private final long offset;
+    private final Record record;
+
+    public LogEntry(long offset, Record record) {
+        this.offset = offset;
+        this.record = record;
+    }
+
+    public long offset() {
+        return this.offset;
+    }
+
+    public Record record() {
+        return this.record;
+    }
+
+    @Override
+    public String toString() {
+        return "LogEntry(" + offset + ", " + record + ")";
+    }
+    
+    public int size() {
+        return record.size() + Records.LOG_OVERHEAD;
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/MemoryRecords.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/MemoryRecords.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/MemoryRecords.java
new file mode 100644
index 0000000..15e98ed
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/MemoryRecords.java
@@ -0,0 +1,280 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.record;
+
+import org.apache.flink.kafka_backport.common.KafkaException;
+import org.apache.flink.kafka_backport.common.utils.AbstractIterator;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.GatheringByteChannel;
+import java.util.Iterator;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A {@link Records} implementation backed by a ByteBuffer.
+ */
+public class MemoryRecords implements Records {
+
+    private final Compressor compressor;
+    private final int capacity;
+    private final int sizeLimit;
+    private ByteBuffer buffer;
+    private boolean writable;
+
+    // Construct a writable memory records
+    private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int sizeLimit) {
+        this.writable = writable;
+        this.capacity = buffer.capacity();
+        this.sizeLimit = sizeLimit;
+        if (this.writable) {
+            this.buffer = null;
+            this.compressor = new Compressor(buffer, type);
+        } else {
+            this.buffer = buffer;
+            this.compressor = null;
+        }
+    }
+
+    public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int capacity) {
+        return new MemoryRecords(buffer, type, true, capacity);
+    }
+
+    public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) {
+        return emptyRecords(buffer, type, buffer.capacity());
+    }
+
+    public static MemoryRecords readableRecords(ByteBuffer buffer) {
+        return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity());
+    }
+
+    /**
+     * Append the given record and offset to the buffer
+     */
+    public void append(long offset, Record record) {
+        if (!writable)
+            throw new IllegalStateException("Memory records is not writable");
+
+        int size = record.size();
+        compressor.putLong(offset);
+        compressor.putInt(size);
+        compressor.put(record.buffer());
+        compressor.recordWritten(size + LOG_OVERHEAD);
+        record.buffer().rewind();
+    }
+
+    /**
+     * Append a new record and offset to the buffer
+     */
+    public void append(long offset, byte[] key, byte[] value) {
+        if (!writable)
+            throw new IllegalStateException("Memory records is not writable");
+
+        int size = Record.recordSize(key, value);
+        compressor.putLong(offset);
+        compressor.putInt(size);
+        compressor.putRecord(key, value);
+        compressor.recordWritten(size + LOG_OVERHEAD);
+    }
+
+    /**
+     * Check if we have room for a new record containing the given key/value pair
+     * 
+     * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
+     * accurate if compression is really used. When this happens, the following append may cause dynamic buffer
+     * re-allocation in the underlying byte buffer stream.
+     * 
+     * Also note that besides the records' capacity, there is also a size limit for the batch. This size limit may be
+     * smaller than the capacity (e.g. when appending a single message whose size is larger than the batch size, the
+     * capacity will be the message size, but the size limit will still be the batch size), and when the records' size
+     * has exceed this limit we also mark this record as full.
+     */
+    public boolean hasRoomFor(byte[] key, byte[] value) {
+        return this.writable && this.capacity >= this.compressor.estimatedBytesWritten() + LOG_OVERHEAD +
+                                                 Record.recordSize(key, value) &&
+               this.sizeLimit >= this.compressor.estimatedBytesWritten();
+    }
+
+    public boolean isFull() {
+        return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten() ||
+               this.sizeLimit <= this.compressor.estimatedBytesWritten();
+    }
+
+    /**
+     * Close this batch for no more appends
+     */
+    public void close() {
+        if (writable) {
+            compressor.close();
+            writable = false;
+            buffer = compressor.buffer();
+        }
+    }
+
+    /** Write the records in this set to the given channel */
+    public int writeTo(GatheringByteChannel channel) throws IOException {
+        return channel.write(buffer);
+    }
+
+    /**
+     * The size of this record set
+     */
+    public int sizeInBytes() {
+        return compressor.buffer().position();
+    }
+
+    /**
+     * The compression rate of this record set
+     */
+    public double compressionRate() {
+        if (compressor == null)
+            return 1.0;
+        else
+            return compressor.compressionRate();
+    }
+
+    /**
+     * Return the capacity of the buffer
+     */
+    public int capacity() {
+        return this.capacity;
+    }
+
+    /**
+     * Get the byte buffer that backs this records instance
+     */
+    public ByteBuffer buffer() {
+        return buffer.duplicate();
+    }
+
+    /**
+     * Return a flipped duplicate of the closed buffer to reading records
+     */
+    public ByteBuffer flip() {
+        if (writable)
+            throw new IllegalStateException("The memory records need to be closed for write before rewinding for read");
+
+        return (ByteBuffer) buffer.flip();
+    }
+
+    @Override
+    public Iterator<LogEntry> iterator() {
+        ByteBuffer copy = this.buffer.duplicate();
+        return new RecordsIterator(copy, CompressionType.NONE, false);
+    }
+    
+    @Override
+    public String toString() {
+        Iterator<LogEntry> iter = iterator();
+        StringBuilder builder = new StringBuilder();
+        builder.append('[');
+        while (iter.hasNext()) {
+            LogEntry entry = iter.next();
+            builder.append('(');
+            builder.append("offset=");
+            builder.append(entry.offset());
+            builder.append(",");
+            builder.append("record=");
+            builder.append(entry.record());
+            builder.append(")");
+        }
+        builder.append(']');
+        return builder.toString();
+    }
+
+    public static class RecordsIterator extends AbstractIterator<LogEntry> {
+        private final ByteBuffer buffer;
+        private final DataInputStream stream;
+        private final CompressionType type;
+        private final boolean shallow;
+        private RecordsIterator innerIter;
+
+        public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) {
+            this.type = type;
+            this.buffer = buffer;
+            this.shallow = shallow;
+            this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type);
+        }
+
+        /*
+         * Read the next record from the buffer.
+         * 
+         * Note that in the compressed message set, each message value size is set as the size of the un-compressed
+         * version of the message value, so when we do de-compression allocating an array of the specified size for
+         * reading compressed value data is sufficient.
+         */
+        @Override
+        protected LogEntry makeNext() {
+            if (innerDone()) {
+                try {
+                    // read the offset
+                    long offset = stream.readLong();
+                    // read record size
+                    int size = stream.readInt();
+                    if (size < 0)
+                        throw new IllegalStateException("Record with size " + size);
+                    // read the record, if compression is used we cannot depend on size
+                    // and hence has to do extra copy
+                    ByteBuffer rec;
+                    if (type == CompressionType.NONE) {
+                        rec = buffer.slice();
+                        int newPos = buffer.position() + size;
+                        if (newPos > buffer.limit())
+                            return allDone();
+                        buffer.position(newPos);
+                        rec.limit(size);
+                    } else {
+                        byte[] recordBuffer = new byte[size];
+                        stream.readFully(recordBuffer, 0, size);
+                        rec = ByteBuffer.wrap(recordBuffer);
+                    }
+                    LogEntry entry = new LogEntry(offset, new Record(rec));
+
+                    // decide whether to go shallow or deep iteration if it is compressed
+                    CompressionType compression = entry.record().compressionType();
+                    if (compression == CompressionType.NONE || shallow) {
+                        return entry;
+                    } else {
+                        // init the inner iterator with the value payload of the message,
+                        // which will de-compress the payload to a set of messages;
+                        // since we assume nested compression is not allowed, the deep iterator
+                        // would not try to further decompress underlying messages
+                        ByteBuffer value = entry.record().value();
+                        innerIter = new RecordsIterator(value, compression, true);
+                        return innerIter.next();
+                    }
+                } catch (EOFException e) {
+                    return allDone();
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            } else {
+                return innerIter.next();
+            }
+        }
+
+        private boolean innerDone() {
+            return innerIter == null || !innerIter.hasNext();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Record.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Record.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Record.java
new file mode 100644
index 0000000..f71900c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Record.java
@@ -0,0 +1,352 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.record;
+
+import org.apache.flink.kafka_backport.common.utils.Crc32;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A record: a serialized key and value along with the associated CRC and other fields
+ */
+public final class Record {
+
+    /**
+     * The current offset and size for all the fixed-length fields
+     */
+    public static final int CRC_OFFSET = 0;
+    public static final int CRC_LENGTH = 4;
+    public static final int MAGIC_OFFSET = CRC_OFFSET + CRC_LENGTH;
+    public static final int MAGIC_LENGTH = 1;
+    public static final int ATTRIBUTES_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
+    public static final int ATTRIBUTE_LENGTH = 1;
+    public static final int KEY_SIZE_OFFSET = ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH;
+    public static final int KEY_SIZE_LENGTH = 4;
+    public static final int KEY_OFFSET = KEY_SIZE_OFFSET + KEY_SIZE_LENGTH;
+    public static final int VALUE_SIZE_LENGTH = 4;
+
+    /**
+     * The size for the record header
+     */
+    public static final int HEADER_SIZE = CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH;
+
+    /**
+     * The amount of overhead bytes in a record
+     */
+    public static final int RECORD_OVERHEAD = HEADER_SIZE + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
+
+    /**
+     * The current "magic" value
+     */
+    public static final byte CURRENT_MAGIC_VALUE = 0;
+
+    /**
+     * Specifies the mask for the compression code. 3 bits to hold the compression codec. 0 is reserved to indicate no
+     * compression
+     */
+    public static final int COMPRESSION_CODEC_MASK = 0x07;
+
+    /**
+     * Compression code for uncompressed records
+     */
+    public static final int NO_COMPRESSION = 0;
+
+    private final ByteBuffer buffer;
+
+    public Record(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    /**
+     * A constructor to create a LogRecord. If the record's compression type is not none, then
+     * its value payload should be already compressed with the specified type; the constructor
+     * would always write the value payload as is and will not do the compression itself.
+     * 
+     * @param key The key of the record (null, if none)
+     * @param value The record value
+     * @param type The compression type used on the contents of the record (if any)
+     * @param valueOffset The offset into the payload array used to extract payload
+     * @param valueSize The size of the payload to use
+     */
+    public Record(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
+        this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length,
+                value == null ? 0 : valueSize >= 0 ? valueSize : value.length - valueOffset)));
+        write(this.buffer, key, value, type, valueOffset, valueSize);
+        this.buffer.rewind();
+    }
+
+    public Record(byte[] key, byte[] value, CompressionType type) {
+        this(key, value, type, 0, -1);
+    }
+
+    public Record(byte[] value, CompressionType type) {
+        this(null, value, type);
+    }
+
+    public Record(byte[] key, byte[] value) {
+        this(key, value, CompressionType.NONE);
+    }
+
+    public Record(byte[] value) {
+        this(null, value, CompressionType.NONE);
+    }
+
+    // Write a record to the buffer, if the record's compression type is none, then
+    // its value payload should be already compressed with the specified type
+    public static void write(ByteBuffer buffer, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
+        // construct the compressor with compression type none since this function will not do any
+        //compression according to the input type, it will just write the record's payload as is
+        Compressor compressor = new Compressor(buffer, CompressionType.NONE, buffer.capacity());
+        compressor.putRecord(key, value, type, valueOffset, valueSize);
+    }
+
+    public static void write(Compressor compressor, long crc, byte attributes, byte[] key, byte[] value, int valueOffset, int valueSize) {
+        // write crc
+        compressor.putInt((int) (crc & 0xffffffffL));
+        // write magic value
+        compressor.putByte(CURRENT_MAGIC_VALUE);
+        // write attributes
+        compressor.putByte(attributes);
+        // write the key
+        if (key == null) {
+            compressor.putInt(-1);
+        } else {
+            compressor.putInt(key.length);
+            compressor.put(key, 0, key.length);
+        }
+        // write the value
+        if (value == null) {
+            compressor.putInt(-1);
+        } else {
+            int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
+            compressor.putInt(size);
+            compressor.put(value, valueOffset, size);
+        }
+    }
+
+    public static int recordSize(byte[] key, byte[] value) {
+        return recordSize(key == null ? 0 : key.length, value == null ? 0 : value.length);
+    }
+
+    public static int recordSize(int keySize, int valueSize) {
+        return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
+    }
+
+    public ByteBuffer buffer() {
+        return this.buffer;
+    }
+
+    public static byte computeAttributes(CompressionType type) {
+        byte attributes = 0;
+        if (type.id > 0)
+            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
+        return attributes;
+    }
+
+    /**
+     * Compute the checksum of the record from the record contents
+     */
+    public static long computeChecksum(ByteBuffer buffer, int position, int size) {
+        Crc32 crc = new Crc32();
+        crc.update(buffer.array(), buffer.arrayOffset() + position, size);
+        return crc.getValue();
+    }
+
+    /**
+     * Compute the checksum of the record from the attributes, key and value payloads
+     */
+    public static long computeChecksum(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
+        Crc32 crc = new Crc32();
+        crc.update(CURRENT_MAGIC_VALUE);
+        byte attributes = 0;
+        if (type.id > 0)
+            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
+        crc.update(attributes);
+        // update for the key
+        if (key == null) {
+            crc.updateInt(-1);
+        } else {
+            crc.updateInt(key.length);
+            crc.update(key, 0, key.length);
+        }
+        // update for the value
+        if (value == null) {
+            crc.updateInt(-1);
+        } else {
+            int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
+            crc.updateInt(size);
+            crc.update(value, valueOffset, size);
+        }
+        return crc.getValue();
+    }
+
+
+    /**
+     * Compute the checksum of the record from the record contents
+     */
+    public long computeChecksum() {
+        return computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
+    }
+
+    /**
+     * Retrieve the previously computed CRC for this record
+     */
+    public long checksum() {
+        return Utils.readUnsignedInt(buffer, CRC_OFFSET);
+    }
+
+    /**
+     * Returns true if the crc stored with the record matches the crc computed off the record contents
+     */
+    public boolean isValid() {
+        return checksum() == computeChecksum();
+    }
+
+    /**
+     * Throw an InvalidRecordException if isValid is false for this record
+     */
+    public void ensureValid() {
+        if (!isValid())
+            throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
+                                             + ", computed crc = "
+                                             + computeChecksum()
+                                             + ")");
+    }
+
+    /**
+     * The complete serialized size of this record in bytes (including crc, header attributes, etc)
+     */
+    public int size() {
+        return buffer.limit();
+    }
+
+    /**
+     * The length of the key in bytes
+     */
+    public int keySize() {
+        return buffer.getInt(KEY_SIZE_OFFSET);
+    }
+
+    /**
+     * Does the record have a key?
+     */
+    public boolean hasKey() {
+        return keySize() >= 0;
+    }
+
+    /**
+     * The position where the value size is stored
+     */
+    private int valueSizeOffset() {
+        return KEY_OFFSET + Math.max(0, keySize());
+    }
+
+    /**
+     * The length of the value in bytes
+     */
+    public int valueSize() {
+        return buffer.getInt(valueSizeOffset());
+    }
+
+    /**
+     * The magic version of this record
+     */
+    public byte magic() {
+        return buffer.get(MAGIC_OFFSET);
+    }
+
+    /**
+     * The attributes stored with this record
+     */
+    public byte attributes() {
+        return buffer.get(ATTRIBUTES_OFFSET);
+    }
+
+    /**
+     * The compression type used with this record
+     */
+    public CompressionType compressionType() {
+        return CompressionType.forId(buffer.get(ATTRIBUTES_OFFSET) & COMPRESSION_CODEC_MASK);
+    }
+
+    /**
+     * A ByteBuffer containing the value of this record
+     */
+    public ByteBuffer value() {
+        return sliceDelimited(valueSizeOffset());
+    }
+
+    /**
+     * A ByteBuffer containing the message key
+     */
+    public ByteBuffer key() {
+        return sliceDelimited(KEY_SIZE_OFFSET);
+    }
+
+    /**
+     * Read a size-delimited byte buffer starting at the given offset
+     */
+    private ByteBuffer sliceDelimited(int start) {
+        int size = buffer.getInt(start);
+        if (size < 0) {
+            return null;
+        } else {
+            ByteBuffer b = buffer.duplicate();
+            b.position(start + 4);
+            b = b.slice();
+            b.limit(size);
+            b.rewind();
+            return b;
+        }
+    }
+
+    public String toString() {
+        return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, key = %d bytes, value = %d bytes)",
+                magic(),
+                attributes(),
+                compressionType(),
+                checksum(),
+                key() == null ? 0 : key().limit(),
+                value() == null ? 0 : value().limit());
+    }
+
+    public boolean equals(Object other) {
+        if (this == other)
+            return true;
+        if (other == null)
+            return false;
+        if (!other.getClass().equals(Record.class))
+            return false;
+        Record record = (Record) other;
+        return this.buffer.equals(record.buffer);
+    }
+
+    public int hashCode() {
+        return buffer.hashCode();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Records.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Records.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Records.java
new file mode 100644
index 0000000..433748b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/record/Records.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.kafka_backport.common.record;
+
+import java.io.IOException;
+import java.nio.channels.GatheringByteChannel;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A binary format which consists of a 4 byte size, an 8 byte offset, and the record bytes. See {@link MemoryRecords}
+ * for the in-memory representation.
+ */
+public interface Records extends Iterable<LogEntry> {
+
+    int SIZE_LENGTH = 4;
+    int OFFSET_LENGTH = 8;
+    int LOG_OVERHEAD = SIZE_LENGTH + OFFSET_LENGTH;
+
+    /**
+     * Write these records to the given channel
+     * @param channel The channel to write to
+     * @return The number of bytes written
+     * @throws IOException If the write fails.
+     */
+    public int writeTo(GatheringByteChannel channel) throws IOException;
+
+    /**
+     * The size of these records in bytes
+     */
+    public int sizeInBytes();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequest.java
new file mode 100644
index 0000000..e888d1e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequest.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public abstract class AbstractRequest extends AbstractRequestResponse {
+
+    public AbstractRequest(Struct struct) {
+        super(struct);
+    }
+
+    /**
+     * Get an error response for a request for a given api version
+     */
+    public abstract AbstractRequestResponse getErrorResponse(int versionId, Throwable e);
+
+    /**
+     * Factory method for getting a request object based on ApiKey ID and a buffer
+     */
+    public static AbstractRequest getRequest(int requestId, int versionId, ByteBuffer buffer) {
+        switch (ApiKeys.forId(requestId)) {
+            case PRODUCE:
+                return ProduceRequest.parse(buffer, versionId);
+            case FETCH:
+                return FetchRequest.parse(buffer, versionId);
+            case LIST_OFFSETS:
+                return ListOffsetRequest.parse(buffer, versionId);
+            case METADATA:
+                return MetadataRequest.parse(buffer, versionId);
+            case OFFSET_COMMIT:
+                return OffsetCommitRequest.parse(buffer, versionId);
+            case OFFSET_FETCH:
+                return OffsetFetchRequest.parse(buffer, versionId);
+            case CONSUMER_METADATA:
+                return ConsumerMetadataRequest.parse(buffer, versionId);
+            case JOIN_GROUP:
+                return JoinGroupRequest.parse(buffer, versionId);
+            case HEARTBEAT:
+                return HeartbeatRequest.parse(buffer, versionId);
+            default:
+                return null;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequestResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequestResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequestResponse.java
new file mode 100644
index 0000000..002e8db
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/AbstractRequestResponse.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public abstract class AbstractRequestResponse {
+    protected final Struct struct;
+
+
+    public AbstractRequestResponse(Struct struct) {
+        this.struct = struct;
+    }
+
+    public Struct toStruct() {
+        return struct;
+    }
+
+    /**
+     * Get the serialized size of this object
+     */
+    public int sizeOf() {
+        return struct.sizeOf();
+    }
+
+    /**
+     * Write this object to a buffer
+     */
+    public void writeTo(ByteBuffer buffer) {
+        struct.writeTo(buffer);
+    }
+
+    @Override
+    public String toString() {
+        return struct.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        return struct.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        AbstractRequestResponse other = (AbstractRequestResponse) obj;
+        return struct.equals(other.struct);
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataRequest.java
new file mode 100644
index 0000000..e4f5e90
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataRequest.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.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+import org.apache.flink.kafka_backport.common.protocol.Errors;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ConsumerMetadataRequest extends AbstractRequest {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+
+    private final String groupId;
+
+    public ConsumerMetadataRequest(String groupId) {
+        super(new Struct(CURRENT_SCHEMA));
+
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        this.groupId = groupId;
+    }
+
+    public ConsumerMetadataRequest(Struct struct) {
+        super(struct);
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        switch (versionId) {
+            case 0:
+                return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode());
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CONSUMER_METADATA.id)));
+        }
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public static ConsumerMetadataRequest parse(ByteBuffer buffer, int versionId) {
+        return new ConsumerMetadataRequest(ProtoUtils.parseRequest(ApiKeys.CONSUMER_METADATA.id, versionId, buffer));
+    }
+
+    public static ConsumerMetadataRequest parse(ByteBuffer buffer) {
+        return new ConsumerMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataResponse.java
new file mode 100644
index 0000000..3688eda
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/common/requests/ConsumerMetadataResponse.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.common.requests;
+
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.protocol.ProtoUtils;
+import org.apache.flink.kafka_backport.common.protocol.types.Schema;
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+
+import java.nio.ByteBuffer;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ConsumerMetadataResponse extends AbstractRequestResponse {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id);
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final String COORDINATOR_KEY_NAME = "coordinator";
+
+    // coordinator level field names
+    private static final String NODE_ID_KEY_NAME = "node_id";
+    private static final String HOST_KEY_NAME = "host";
+    private static final String PORT_KEY_NAME = "port";
+
+    private final short errorCode;
+    private final Node node;
+
+    public ConsumerMetadataResponse(short errorCode, Node node) {
+        super(new Struct(CURRENT_SCHEMA));
+        struct.set(ERROR_CODE_KEY_NAME, errorCode);
+        Struct coordinator = struct.instance(COORDINATOR_KEY_NAME);
+        coordinator.set(NODE_ID_KEY_NAME, node.id());
+        coordinator.set(HOST_KEY_NAME, node.host());
+        coordinator.set(PORT_KEY_NAME, node.port());
+        struct.set(COORDINATOR_KEY_NAME, coordinator);
+        this.errorCode = errorCode;
+        this.node = node;
+    }
+
+    public ConsumerMetadataResponse(Struct struct) {
+        super(struct);
+        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+        Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME);
+        int nodeId = broker.getInt(NODE_ID_KEY_NAME);
+        String host = broker.getString(HOST_KEY_NAME);
+        int port = broker.getInt(PORT_KEY_NAME);
+        node = new Node(nodeId, host, port);
+    }
+
+    public short errorCode() {
+        return errorCode;
+    }
+
+    public Node node() {
+        return node;
+    }
+
+    public static ConsumerMetadataResponse parse(ByteBuffer buffer) {
+        return new ConsumerMetadataResponse((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}
\ No newline at end of file


[24/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
[FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'


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

Branch: refs/heads/master
Commit: 33f4c818dd81d259f5b6c06f5caeda0376c40750
Parents: fb5aac2
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Aug 11 16:48:26 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Aug 27 12:40:38 2015 +0200

----------------------------------------------------------------------
 .../kafka_backport/clients/ClientRequest.java   |   75 ++
 .../kafka_backport/clients/ClientResponse.java  |   87 ++
 .../kafka_backport/clients/ClientUtils.java     |   71 ++
 .../clients/ClusterConnectionStates.java        |  170 +++
 .../clients/CommonClientConfigs.java            |   69 ++
 .../kafka_backport/clients/ConnectionState.java |   29 +
 .../clients/InFlightRequests.java               |  135 +++
 .../kafka_backport/clients/KafkaClient.java     |  143 +++
 .../flink/kafka_backport/clients/Metadata.java  |  195 +++
 .../kafka_backport/clients/NetworkClient.java   |  528 ++++++++
 .../clients/RequestCompletionHandler.java       |   32 +
 .../clients/consumer/CommitType.java            |   26 +
 .../clients/consumer/Consumer.java              |  135 +++
 .../consumer/ConsumerCommitCallback.java        |   42 +
 .../clients/consumer/ConsumerConfig.java        |  334 ++++++
 .../consumer/ConsumerRebalanceCallback.java     |  104 ++
 .../clients/consumer/ConsumerRecord.java        |   93 ++
 .../clients/consumer/ConsumerRecords.java       |  126 ++
 .../consumer/ConsumerWakeupException.java       |   29 +
 .../clients/consumer/KafkaConsumer.java         | 1130 ++++++++++++++++++
 .../clients/consumer/MockConsumer.java          |  209 ++++
 .../consumer/NoOffsetForPartitionException.java |   38 +
 .../clients/consumer/OffsetResetStrategy.java   |   26 +
 .../internals/ConsumerNetworkClient.java        |  296 +++++
 .../clients/consumer/internals/Coordinator.java |  791 ++++++++++++
 .../clients/consumer/internals/DelayedTask.java |   24 +
 .../consumer/internals/DelayedTaskQueue.java    |   96 ++
 .../clients/consumer/internals/Fetcher.java     |  506 ++++++++
 .../clients/consumer/internals/Heartbeat.java   |   83 ++
 .../internals/NoAvailableBrokersException.java  |   32 +
 .../NoOpConsumerRebalanceCallback.java          |   39 +
 .../consumer/internals/RequestFuture.java       |  211 ++++
 .../internals/RequestFutureAdapter.java         |   37 +
 .../internals/RequestFutureListener.java        |   32 +
 .../consumer/internals/SendFailedException.java |   36 +
 .../internals/StaleMetadataException.java       |   31 +
 .../consumer/internals/SubscriptionState.java   |  242 ++++
 .../flink/kafka_backport/common/Cluster.java    |  203 ++++
 .../kafka_backport/common/Configurable.java     |   40 +
 .../kafka_backport/common/KafkaException.java   |   51 +
 .../flink/kafka_backport/common/Metric.java     |   43 +
 .../flink/kafka_backport/common/MetricName.java |  194 +++
 .../flink/kafka_backport/common/Node.java       |  113 ++
 .../kafka_backport/common/PartitionInfo.java    |  104 ++
 .../kafka_backport/common/TopicPartition.java   |   89 ++
 .../common/config/AbstractConfig.java           |  185 +++
 .../kafka_backport/common/config/ConfigDef.java |  456 +++++++
 .../common/config/ConfigException.java          |   49 +
 .../common/errors/ApiException.java             |   60 +
 ...onsumerCoordinatorNotAvailableException.java |   49 +
 .../common/errors/CorruptRecordException.java   |   48 +
 .../common/errors/DisconnectException.java      |   47 +
 .../errors/IllegalGenerationException.java      |   42 +
 .../common/errors/InterruptException.java       |   48 +
 .../common/errors/InvalidMetadataException.java |   48 +
 .../errors/InvalidRequiredAcksException.java    |   34 +
 .../common/errors/InvalidTopicException.java    |   47 +
 .../errors/LeaderNotAvailableException.java     |   36 +
 .../common/errors/NetworkException.java         |   48 +
 .../NotCoordinatorForConsumerException.java     |   49 +
 .../NotEnoughReplicasAfterAppendException.java  |   39 +
 .../errors/NotEnoughReplicasException.java      |   49 +
 .../errors/NotLeaderForPartitionException.java  |   47 +
 .../errors/OffsetLoadInProgressException.java   |   49 +
 .../common/errors/OffsetMetadataTooLarge.java   |   46 +
 .../errors/OffsetOutOfRangeException.java       |   47 +
 .../errors/RecordBatchTooLargeException.java    |   48 +
 .../common/errors/RecordTooLargeException.java  |   47 +
 .../common/errors/RetriableException.java       |   46 +
 .../common/errors/SerializationException.java   |   55 +
 .../common/errors/TimeoutException.java         |   47 +
 .../errors/UnknownConsumerIdException.java      |   42 +
 .../common/errors/UnknownServerException.java   |   48 +
 .../UnknownTopicOrPartitionException.java       |   46 +
 .../common/metrics/CompoundStat.java            |   61 +
 .../common/metrics/JmxReporter.java             |  225 ++++
 .../common/metrics/KafkaMetric.java             |   74 ++
 .../common/metrics/Measurable.java              |   37 +
 .../common/metrics/MeasurableStat.java          |   38 +
 .../common/metrics/MetricConfig.java            |   96 ++
 .../kafka_backport/common/metrics/Metrics.java  |  211 ++++
 .../common/metrics/MetricsReporter.java         |   50 +
 .../kafka_backport/common/metrics/Quota.java    |   61 +
 .../common/metrics/QuotaViolationException.java |   41 +
 .../kafka_backport/common/metrics/Sensor.java   |  182 +++
 .../kafka_backport/common/metrics/Stat.java     |   41 +
 .../common/metrics/stats/Avg.java               |   54 +
 .../common/metrics/stats/Count.java             |   50 +
 .../common/metrics/stats/Histogram.java         |  166 +++
 .../common/metrics/stats/Max.java               |   50 +
 .../common/metrics/stats/Min.java               |   50 +
 .../common/metrics/stats/Percentile.java        |   49 +
 .../common/metrics/stats/Percentiles.java       |  125 ++
 .../common/metrics/stats/Rate.java              |  115 ++
 .../common/metrics/stats/SampledStat.java       |  139 +++
 .../common/metrics/stats/Total.java             |   52 +
 .../common/network/ByteBufferReceive.java       |   66 +
 .../common/network/ByteBufferSend.java          |   71 ++
 .../common/network/InvalidReceiveException.java |   39 +
 .../common/network/MultiSend.java               |  108 ++
 .../common/network/NetworkReceive.java          |  128 ++
 .../common/network/NetworkSend.java             |   51 +
 .../kafka_backport/common/network/Receive.java  |   54 +
 .../common/network/Selectable.java              |  115 ++
 .../kafka_backport/common/network/Selector.java |  664 ++++++++++
 .../kafka_backport/common/network/Send.java     |   56 +
 .../kafka_backport/common/protocol/ApiKeys.java |   75 ++
 .../kafka_backport/common/protocol/Errors.java  |  172 +++
 .../common/protocol/ProtoUtils.java             |   74 ++
 .../common/protocol/Protocol.java               |  474 ++++++++
 .../common/protocol/SecurityProtocol.java       |   72 ++
 .../common/protocol/types/ArrayOf.java          |   88 ++
 .../common/protocol/types/Field.java            |   78 ++
 .../common/protocol/types/Schema.java           |  168 +++
 .../common/protocol/types/SchemaException.java  |   41 +
 .../common/protocol/types/Struct.java           |  338 ++++++
 .../common/protocol/types/Type.java             |  259 ++++
 .../common/record/ByteBufferInputStream.java    |   58 +
 .../common/record/ByteBufferOutputStream.java   |   66 +
 .../common/record/CompressionType.java          |   72 ++
 .../common/record/Compressor.java               |  279 +++++
 .../common/record/InvalidRecordException.java   |   36 +
 .../common/record/KafkaLZ4BlockInputStream.java |  236 ++++
 .../record/KafkaLZ4BlockOutputStream.java       |  400 +++++++
 .../kafka_backport/common/record/LogEntry.java  |   57 +
 .../common/record/MemoryRecords.java            |  280 +++++
 .../kafka_backport/common/record/Record.java    |  352 ++++++
 .../kafka_backport/common/record/Records.java   |   54 +
 .../common/requests/AbstractRequest.java        |   71 ++
 .../requests/AbstractRequestResponse.java       |   75 ++
 .../requests/ConsumerMetadataRequest.java       |   74 ++
 .../requests/ConsumerMetadataResponse.java      |   79 ++
 .../common/requests/FetchRequest.java           |  174 +++
 .../common/requests/FetchResponse.java          |  134 +++
 .../common/requests/HeartbeatRequest.java       |   90 ++
 .../common/requests/HeartbeatResponse.java      |   64 +
 .../common/requests/JoinGroupRequest.java       |  121 ++
 .../common/requests/JoinGroupResponse.java      |  122 ++
 .../common/requests/ListOffsetRequest.java      |  151 +++
 .../common/requests/ListOffsetResponse.java     |  127 ++
 .../common/requests/MetadataRequest.java        |   89 ++
 .../common/requests/MetadataResponse.java       |  186 +++
 .../common/requests/OffsetCommitRequest.java    |  275 +++++
 .../common/requests/OffsetCommitResponse.java   |  109 ++
 .../common/requests/OffsetFetchRequest.java     |  132 ++
 .../common/requests/OffsetFetchResponse.java    |  135 +++
 .../common/requests/ProduceRequest.java         |  141 +++
 .../common/requests/ProduceResponse.java        |  131 ++
 .../common/requests/RequestHeader.java          |   89 ++
 .../common/requests/RequestSend.java            |   64 +
 .../common/requests/ResponseHeader.java         |   62 +
 .../common/requests/ResponseSend.java           |   50 +
 .../serialization/ByteArrayDeserializer.java    |   43 +
 .../serialization/ByteArraySerializer.java      |   43 +
 .../common/serialization/Deserializer.java      |   53 +
 .../serialization/IntegerDeserializer.java      |   53 +
 .../common/serialization/IntegerSerializer.java |   47 +
 .../common/serialization/Serializer.java        |   58 +
 .../serialization/StringDeserializer.java       |   62 +
 .../common/serialization/StringSerializer.java  |   62 +
 .../common/utils/AbstractIterator.java          |   97 ++
 .../common/utils/CollectionUtils.java           |   71 ++
 .../common/utils/CopyOnWriteMap.java            |  151 +++
 .../kafka_backport/common/utils/Crc32.java      |  396 ++++++
 .../common/utils/KafkaThread.java               |   44 +
 .../kafka_backport/common/utils/SystemTime.java |   52 +
 .../flink/kafka_backport/common/utils/Time.java |   48 +
 .../kafka_backport/common/utils/Utils.java      |  506 ++++++++
 .../flink/kafka_backport/package-info.java      |   10 +
 .../streaming/connectors/internals/Fetcher.java |    2 +-
 .../internals/FlinkKafkaConsumerBase.java       |   10 +-
 .../connectors/internals/IncludedFetcher.java   |   12 +-
 .../connectors/internals/LegacyFetcher.java     |    6 +-
 .../kafka/copied/clients/ClientRequest.java     |   66 -
 .../kafka/copied/clients/ClientResponse.java    |   78 --
 .../kafka/copied/clients/ClientUtils.java       |   64 -
 .../copied/clients/ClusterConnectionStates.java |  161 ---
 .../copied/clients/CommonClientConfigs.java     |   60 -
 .../kafka/copied/clients/ConnectionState.java   |   20 -
 .../kafka/copied/clients/InFlightRequests.java  |  126 --
 .../kafka/copied/clients/KafkaClient.java       |  134 ---
 .../apache/kafka/copied/clients/Metadata.java   |  186 ---
 .../kafka/copied/clients/NetworkClient.java     |  519 --------
 .../clients/RequestCompletionHandler.java       |   23 -
 .../copied/clients/consumer/CommitType.java     |   17 -
 .../kafka/copied/clients/consumer/Consumer.java |  126 --
 .../consumer/ConsumerCommitCallback.java        |   33 -
 .../copied/clients/consumer/ConsumerConfig.java |  325 -----
 .../consumer/ConsumerRebalanceCallback.java     |   95 --
 .../copied/clients/consumer/ConsumerRecord.java |   84 --
 .../clients/consumer/ConsumerRecords.java       |  117 --
 .../consumer/ConsumerWakeupException.java       |   20 -
 .../copied/clients/consumer/KafkaConsumer.java  | 1121 -----------------
 .../copied/clients/consumer/MockConsumer.java   |  200 ----
 .../consumer/NoOffsetForPartitionException.java |   29 -
 .../clients/consumer/OffsetResetStrategy.java   |   17 -
 .../internals/ConsumerNetworkClient.java        |  296 -----
 .../clients/consumer/internals/Coordinator.java |  791 ------------
 .../clients/consumer/internals/DelayedTask.java |   24 -
 .../consumer/internals/DelayedTaskQueue.java    |   96 --
 .../clients/consumer/internals/Fetcher.java     |  498 --------
 .../clients/consumer/internals/Heartbeat.java   |   74 --
 .../internals/NoAvailableBrokersException.java  |   23 -
 .../NoOpConsumerRebalanceCallback.java          |   30 -
 .../consumer/internals/RequestFuture.java       |  202 ----
 .../internals/RequestFutureAdapter.java         |   28 -
 .../internals/RequestFutureListener.java        |   23 -
 .../consumer/internals/SendFailedException.java |   27 -
 .../internals/StaleMetadataException.java       |   22 -
 .../consumer/internals/SubscriptionState.java   |  233 ----
 .../org/apache/kafka/copied/common/Cluster.java |  194 ---
 .../kafka/copied/common/Configurable.java       |   31 -
 .../kafka/copied/common/KafkaException.java     |   42 -
 .../org/apache/kafka/copied/common/Metric.java  |   34 -
 .../apache/kafka/copied/common/MetricName.java  |  185 ---
 .../org/apache/kafka/copied/common/Node.java    |  104 --
 .../kafka/copied/common/PartitionInfo.java      |   95 --
 .../kafka/copied/common/TopicPartition.java     |   79 --
 .../copied/common/config/AbstractConfig.java    |  176 ---
 .../kafka/copied/common/config/ConfigDef.java   |  447 -------
 .../copied/common/config/ConfigException.java   |   40 -
 .../copied/common/errors/ApiException.java      |   51 -
 ...onsumerCoordinatorNotAvailableException.java |   40 -
 .../common/errors/CorruptRecordException.java   |   39 -
 .../common/errors/DisconnectException.java      |   39 -
 .../errors/IllegalGenerationException.java      |   33 -
 .../common/errors/InterruptException.java       |   39 -
 .../common/errors/InvalidMetadataException.java |   39 -
 .../errors/InvalidRequiredAcksException.java    |   25 -
 .../common/errors/InvalidTopicException.java    |   38 -
 .../errors/LeaderNotAvailableException.java     |   27 -
 .../copied/common/errors/NetworkException.java  |   39 -
 .../NotCoordinatorForConsumerException.java     |   40 -
 .../NotEnoughReplicasAfterAppendException.java  |   30 -
 .../errors/NotEnoughReplicasException.java      |   40 -
 .../errors/NotLeaderForPartitionException.java  |   38 -
 .../errors/OffsetLoadInProgressException.java   |   40 -
 .../common/errors/OffsetMetadataTooLarge.java   |   37 -
 .../errors/OffsetOutOfRangeException.java       |   38 -
 .../errors/RecordBatchTooLargeException.java    |   39 -
 .../common/errors/RecordTooLargeException.java  |   38 -
 .../common/errors/RetriableException.java       |   37 -
 .../common/errors/SerializationException.java   |   46 -
 .../copied/common/errors/TimeoutException.java  |   38 -
 .../errors/UnknownConsumerIdException.java      |   33 -
 .../common/errors/UnknownServerException.java   |   39 -
 .../UnknownTopicOrPartitionException.java       |   37 -
 .../copied/common/metrics/CompoundStat.java     |   52 -
 .../copied/common/metrics/JmxReporter.java      |  216 ----
 .../copied/common/metrics/KafkaMetric.java      |   65 -
 .../kafka/copied/common/metrics/Measurable.java |   28 -
 .../copied/common/metrics/MeasurableStat.java   |   30 -
 .../copied/common/metrics/MetricConfig.java     |   87 --
 .../kafka/copied/common/metrics/Metrics.java    |  202 ----
 .../copied/common/metrics/MetricsReporter.java  |   41 -
 .../kafka/copied/common/metrics/Quota.java      |   52 -
 .../common/metrics/QuotaViolationException.java |   32 -
 .../kafka/copied/common/metrics/Sensor.java     |  173 ---
 .../kafka/copied/common/metrics/Stat.java       |   32 -
 .../kafka/copied/common/metrics/stats/Avg.java  |   45 -
 .../copied/common/metrics/stats/Count.java      |   41 -
 .../copied/common/metrics/stats/Histogram.java  |  157 ---
 .../kafka/copied/common/metrics/stats/Max.java  |   41 -
 .../kafka/copied/common/metrics/stats/Min.java  |   41 -
 .../copied/common/metrics/stats/Percentile.java |   40 -
 .../common/metrics/stats/Percentiles.java       |  116 --
 .../kafka/copied/common/metrics/stats/Rate.java |  106 --
 .../common/metrics/stats/SampledStat.java       |  130 --
 .../copied/common/metrics/stats/Total.java      |   43 -
 .../common/network/ByteBufferReceive.java       |   57 -
 .../copied/common/network/ByteBufferSend.java   |   62 -
 .../common/network/InvalidReceiveException.java |   30 -
 .../kafka/copied/common/network/MultiSend.java  |  100 --
 .../copied/common/network/NetworkReceive.java   |  119 --
 .../copied/common/network/NetworkSend.java      |   42 -
 .../kafka/copied/common/network/Receive.java    |   45 -
 .../kafka/copied/common/network/Selectable.java |  106 --
 .../kafka/copied/common/network/Selector.java   |  655 ----------
 .../kafka/copied/common/network/Send.java       |   47 -
 .../kafka/copied/common/protocol/ApiKeys.java   |   66 -
 .../kafka/copied/common/protocol/Errors.java    |  163 ---
 .../copied/common/protocol/ProtoUtils.java      |   65 -
 .../kafka/copied/common/protocol/Protocol.java  |  470 --------
 .../common/protocol/SecurityProtocol.java       |   63 -
 .../copied/common/protocol/types/ArrayOf.java   |   79 --
 .../copied/common/protocol/types/Field.java     |   69 --
 .../copied/common/protocol/types/Schema.java    |  159 ---
 .../common/protocol/types/SchemaException.java  |   32 -
 .../copied/common/protocol/types/Struct.java    |  329 -----
 .../copied/common/protocol/types/Type.java      |  250 ----
 .../common/record/ByteBufferInputStream.java    |   49 -
 .../common/record/ByteBufferOutputStream.java   |   57 -
 .../copied/common/record/CompressionType.java   |   63 -
 .../kafka/copied/common/record/Compressor.java  |  270 -----
 .../common/record/InvalidRecordException.java   |   27 -
 .../common/record/KafkaLZ4BlockInputStream.java |  233 ----
 .../record/KafkaLZ4BlockOutputStream.java       |  391 ------
 .../kafka/copied/common/record/LogEntry.java    |   48 -
 .../copied/common/record/MemoryRecords.java     |  271 -----
 .../kafka/copied/common/record/Record.java      |  344 ------
 .../kafka/copied/common/record/Records.java     |   45 -
 .../copied/common/requests/AbstractRequest.java |   62 -
 .../requests/AbstractRequestResponse.java       |   66 -
 .../requests/ConsumerMetadataRequest.java       |   65 -
 .../requests/ConsumerMetadataResponse.java      |   70 --
 .../copied/common/requests/FetchRequest.java    |  165 ---
 .../copied/common/requests/FetchResponse.java   |  125 --
 .../common/requests/HeartbeatRequest.java       |   81 --
 .../common/requests/HeartbeatResponse.java      |   55 -
 .../common/requests/JoinGroupRequest.java       |  112 --
 .../common/requests/JoinGroupResponse.java      |  113 --
 .../common/requests/ListOffsetRequest.java      |  142 ---
 .../common/requests/ListOffsetResponse.java     |  118 --
 .../copied/common/requests/MetadataRequest.java |   80 --
 .../common/requests/MetadataResponse.java       |  177 ---
 .../common/requests/OffsetCommitRequest.java    |  266 -----
 .../common/requests/OffsetCommitResponse.java   |  100 --
 .../common/requests/OffsetFetchRequest.java     |  123 --
 .../common/requests/OffsetFetchResponse.java    |  126 --
 .../copied/common/requests/ProduceRequest.java  |  132 --
 .../copied/common/requests/ProduceResponse.java |  122 --
 .../copied/common/requests/RequestHeader.java   |   80 --
 .../copied/common/requests/RequestSend.java     |   55 -
 .../copied/common/requests/ResponseHeader.java  |   55 -
 .../copied/common/requests/ResponseSend.java    |   41 -
 .../serialization/ByteArrayDeserializer.java    |   34 -
 .../serialization/ByteArraySerializer.java      |   34 -
 .../common/serialization/Deserializer.java      |   44 -
 .../serialization/IntegerDeserializer.java      |   44 -
 .../common/serialization/IntegerSerializer.java |   38 -
 .../copied/common/serialization/Serializer.java |   49 -
 .../serialization/StringDeserializer.java       |   53 -
 .../common/serialization/StringSerializer.java  |   53 -
 .../copied/common/utils/AbstractIterator.java   |   88 --
 .../copied/common/utils/CollectionUtils.java    |   62 -
 .../copied/common/utils/CopyOnWriteMap.java     |  142 ---
 .../apache/kafka/copied/common/utils/Crc32.java |  387 ------
 .../kafka/copied/common/utils/KafkaThread.java  |   35 -
 .../kafka/copied/common/utils/SystemTime.java   |   43 -
 .../apache/kafka/copied/common/utils/Time.java  |   39 -
 .../apache/kafka/copied/common/utils/Utils.java |  497 --------
 tools/maven/suppressions.xml                    |    4 +-
 342 files changed, 20627 insertions(+), 19160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientRequest.java
new file mode 100644
index 0000000..d86ea96
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientRequest.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.kafka_backport.clients;
+
+import org.apache.flink.kafka_backport.common.requests.RequestSend;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A request being sent to the server. This holds both the network send as well as the client-level metadata.
+ */
+public final class ClientRequest {
+
+    private final long createdMs;
+    private final boolean expectResponse;
+    private final RequestSend request;
+    private final RequestCompletionHandler callback;
+
+    /**
+     * @param createdMs The unix timestamp in milliseconds for the time at which this request was created.
+     * @param expectResponse Should we expect a response message or is this request complete once it is sent?
+     * @param request The request
+     * @param callback A callback to execute when the response has been received (or null if no callback is necessary)
+     */
+    public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, RequestCompletionHandler callback) {
+        this.createdMs = createdMs;
+        this.callback = callback;
+        this.request = request;
+        this.expectResponse = expectResponse;
+    }
+
+    @Override
+    public String toString() {
+        return "ClientRequest(expectResponse=" + expectResponse + ", callback=" + callback + ", request=" + request
+                + ")";
+    }
+
+    public boolean expectResponse() {
+        return expectResponse;
+    }
+
+    public RequestSend request() {
+        return request;
+    }
+
+    public boolean hasCallback() {
+        return callback != null;
+    }
+
+    public RequestCompletionHandler callback() {
+        return callback;
+    }
+
+    public long createdTime() {
+        return createdMs;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientResponse.java
new file mode 100644
index 0000000..49a7540
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientResponse.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients;
+
+import org.apache.flink.kafka_backport.common.protocol.types.Struct;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A response from the server. Contains both the body of the response as well as the correlated request that was
+ * originally sent.
+ */
+public class ClientResponse {
+
+    private final long received;
+    private final boolean disconnected;
+    private final ClientRequest request;
+    private final Struct responseBody;
+
+    /**
+     * @param request The original request
+     * @param received The unix timestamp when this response was received
+     * @param disconnected Whether the client disconnected before fully reading a response
+     * @param responseBody The response contents (or null) if we disconnected or no response was expected
+     */
+    public ClientResponse(ClientRequest request, long received, boolean disconnected, Struct responseBody) {
+        super();
+        this.received = received;
+        this.disconnected = disconnected;
+        this.request = request;
+        this.responseBody = responseBody;
+    }
+
+    public long receivedTime() {
+        return received;
+    }
+
+    public boolean wasDisconnected() {
+        return disconnected;
+    }
+
+    public ClientRequest request() {
+        return request;
+    }
+
+    public Struct responseBody() {
+        return responseBody;
+    }
+
+    public boolean hasResponse() {
+        return responseBody != null;
+    }
+
+    public long requestLatencyMs() {
+        return receivedTime() - this.request.createdTime();
+    }
+
+    @Override
+    public String toString() {
+        return "ClientResponse(received=" + received +
+               ", disconnected=" +
+               disconnected +
+               ", request=" +
+               request +
+               ", responseBody=" +
+               responseBody +
+               ")";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientUtils.java
new file mode 100644
index 0000000..1a00a78
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientUtils.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.kafka_backport.clients;
+
+import org.apache.flink.kafka_backport.common.config.ConfigException;
+import org.apache.flink.kafka_backport.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+public class ClientUtils {
+    private static final Logger log = LoggerFactory.getLogger(ClientUtils.class);
+
+    public static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls) {
+        List<InetSocketAddress> addresses = new ArrayList<InetSocketAddress>();
+        for (String url : urls) {
+            if (url != null && url.length() > 0) {
+                String host = Utils.getHost(url);
+                Integer port = Utils.getPort(url);
+                if (host == null || port == null)
+                    throw new ConfigException("Invalid url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
+                try {
+                    InetSocketAddress address = new InetSocketAddress(host, port);
+                    if (address.isUnresolved())
+                        throw new ConfigException("DNS resolution failed for url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
+                    addresses.add(address);
+                } catch (NumberFormatException e) {
+                    throw new ConfigException("Invalid port in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
+                }
+            }
+        }
+        if (addresses.size() < 1)
+            throw new ConfigException("No bootstrap urls given in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
+        return addresses;
+    }
+
+    public static void closeQuietly(Closeable c, String name, AtomicReference<Throwable> firstException) {
+        if (c != null) {
+            try {
+                c.close();
+            } catch (Throwable t) {
+                firstException.compareAndSet(null, t);
+                log.error("Failed to close " + name, t);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClusterConnectionStates.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClusterConnectionStates.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClusterConnectionStates.java
new file mode 100644
index 0000000..395164b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClusterConnectionStates.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients;
+
+import java.util.HashMap;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The state of our connection to each node in the cluster.
+ * 
+ */
+final class ClusterConnectionStates {
+    private final long reconnectBackoffMs;
+    private final Map<String, NodeConnectionState> nodeState;
+
+    public ClusterConnectionStates(long reconnectBackoffMs) {
+        this.reconnectBackoffMs = reconnectBackoffMs;
+        this.nodeState = new HashMap<String, NodeConnectionState>();
+    }
+
+    /**
+     * Return true iff we can currently initiate a new connection. This will be the case if we are not
+     * connected and haven't been connected for at least the minimum reconnection backoff period.
+     * @param id The connection id to check
+     * @param now The current time in MS
+     * @return true if we can initiate a new connection
+     */
+    public boolean canConnect(String id, long now) {
+        NodeConnectionState state = nodeState.get(id);
+        if (state == null)
+            return true;
+        else
+            return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs >= this.reconnectBackoffMs;
+    }
+
+    /**
+     * Return true if we are disconnected from the given node and can't re-establish a connection yet
+     * @param id The connection to check
+     * @param now The current time in ms
+     */
+    public boolean isBlackedOut(String id, long now) {
+        NodeConnectionState state = nodeState.get(id);
+        if (state == null)
+            return false;
+        else
+            return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs < this.reconnectBackoffMs;
+    }
+
+    /**
+     * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
+     * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
+     * connections.
+     * @param id The connection to check
+     * @param now The current time in ms
+     */
+    public long connectionDelay(String id, long now) {
+        NodeConnectionState state = nodeState.get(id);
+        if (state == null) return 0;
+        long timeWaited = now - state.lastConnectAttemptMs;
+        if (state.state == ConnectionState.DISCONNECTED) {
+            return Math.max(this.reconnectBackoffMs - timeWaited, 0);
+        } else {
+            // When connecting or connected, we should be able to delay indefinitely since other events (connection or
+            // data acked) will cause a wakeup once data can be sent.
+            return Long.MAX_VALUE;
+        }
+    }
+
+    /**
+     * Enter the connecting state for the given connection.
+     * @param id The id of the connection
+     * @param now The current time.
+     */
+    public void connecting(String id, long now) {
+        nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now));
+    }
+
+    /**
+     * Return true iff a specific connection is connected
+     * @param id The id of the connection to check
+     */
+    public boolean isConnected(String id) {
+        NodeConnectionState state = nodeState.get(id);
+        return state != null && state.state == ConnectionState.CONNECTED;
+    }
+
+    /**
+     * Return true iff we are in the process of connecting
+     * @param id The id of the connection
+     */
+    public boolean isConnecting(String id) {
+        NodeConnectionState state = nodeState.get(id);
+        return state != null && state.state == ConnectionState.CONNECTING;
+    }
+
+    /**
+     * Enter the connected state for the given connection
+     * @param id The connection identifier
+     */
+    public void connected(String id) {
+        NodeConnectionState nodeState = nodeState(id);
+        nodeState.state = ConnectionState.CONNECTED;
+    }
+
+    /**
+     * Enter the disconnected state for the given node
+     * @param id The connection we have disconnected
+     */
+    public void disconnected(String id) {
+        NodeConnectionState nodeState = nodeState(id);
+        nodeState.state = ConnectionState.DISCONNECTED;
+    }
+    
+    /**
+     * Get the state of a given connection
+     * @param id The id of the connection
+     * @return The state of our connection
+     */
+    public ConnectionState connectionState(String id) {
+        return nodeState(id).state;
+    }
+    
+    /**
+     * Get the state of a given node
+     * @param id The connection to fetch the state for
+     */
+    private NodeConnectionState nodeState(String id) {
+        NodeConnectionState state = this.nodeState.get(id);
+        if (state == null)
+            throw new IllegalStateException("No entry found for connection " + id);
+        return state;
+    }
+    
+    /**
+     * The state of our connection to a node
+     */
+    private static class NodeConnectionState {
+
+        ConnectionState state;
+        long lastConnectAttemptMs;
+
+        public NodeConnectionState(ConnectionState state, long lastConnectAttempt) {
+            this.state = state;
+            this.lastConnectAttemptMs = lastConnectAttempt;
+        }
+
+        public String toString() {
+            return "NodeState(" + state + ", " + lastConnectAttemptMs + ")";
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/CommonClientConfigs.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/CommonClientConfigs.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/CommonClientConfigs.java
new file mode 100644
index 0000000..01e9da2
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/CommonClientConfigs.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.kafka_backport.clients;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * Some configurations shared by both producer and consumer
+ */
+public class CommonClientConfigs {
+    
+    /*
+     * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
+     */
+
+    public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
+    public static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping&mdash;this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form "
+                                                       + "<code>host1:port1,host2:port2,...</code>. Since these servers are just used for the initial connection to "
+                                                       + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of "
+                                                       + "servers (you may want more than one, though, in case a server is down).";
+    
+    public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms";
+    public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions.";
+    
+    public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes";
+    public static final String SEND_BUFFER_DOC = "The size of the TCP send buffer (SO_SNDBUF) to use when sending data.";
+
+    public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes";
+    public static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer (SO_RCVBUF) to use when reading data.";
+
+    public static final String CLIENT_ID_CONFIG = "client.id";
+    public static final String CLIENT_ID_DOC = "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging.";
+
+    public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms";
+    public static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all requests sent by the consumer to the broker.";
+
+    public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms";
+    public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop.";
+    
+    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms";
+    public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The number of samples maintained to compute metrics.";
+
+    public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples";
+    public static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics.";
+
+    public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters";
+    public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics.";
+
+    public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = "connections.max.idle.ms";
+    public static final String CONNECTIONS_MAX_IDLE_MS_DOC = "Close idle connections after the number of milliseconds specified by this config.";
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ConnectionState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ConnectionState.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ConnectionState.java
new file mode 100644
index 0000000..34fd7e2
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ConnectionState.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.kafka_backport.clients;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The states of a node connection
+ */
+public enum ConnectionState {
+    DISCONNECTED, CONNECTING, CONNECTED
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/InFlightRequests.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/InFlightRequests.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/InFlightRequests.java
new file mode 100644
index 0000000..9fc489b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/InFlightRequests.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.kafka_backport.clients;
+
+import java.util.ArrayDeque;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.Map;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The set of requests which have been sent or are being sent but haven't yet received a response
+ */
+final class InFlightRequests {
+
+    private final int maxInFlightRequestsPerConnection;
+    private final Map<String, Deque<ClientRequest>> requests = new HashMap<String, Deque<ClientRequest>>();
+
+    public InFlightRequests(int maxInFlightRequestsPerConnection) {
+        this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection;
+    }
+
+    /**
+     * Add the given request to the queue for the connection it was directed to
+     */
+    public void add(ClientRequest request) {
+        Deque<ClientRequest> reqs = this.requests.get(request.request().destination());
+        if (reqs == null) {
+            reqs = new ArrayDeque<ClientRequest>();
+            this.requests.put(request.request().destination(), reqs);
+        }
+        reqs.addFirst(request);
+    }
+
+    /**
+     * Get the request queue for the given node
+     */
+    private Deque<ClientRequest> requestQueue(String node) {
+        Deque<ClientRequest> reqs = requests.get(node);
+        if (reqs == null || reqs.isEmpty())
+            throw new IllegalStateException("Response from server for which there are no in-flight requests.");
+        return reqs;
+    }
+
+    /**
+     * Get the oldest request (the one that that will be completed next) for the given node
+     */
+    public ClientRequest completeNext(String node) {
+        return requestQueue(node).pollLast();
+    }
+
+    /**
+     * Get the last request we sent to the given node (but don't remove it from the queue)
+     * @param node The node id
+     */
+    public ClientRequest lastSent(String node) {
+        return requestQueue(node).peekFirst();
+    }
+
+    /**
+     * Complete the last request that was sent to a particular node.
+     * @param node The node the request was sent to
+     * @return The request
+     */
+    public ClientRequest completeLastSent(String node) {
+        return requestQueue(node).pollFirst();
+    }
+
+    /**
+     * Can we send more requests to this node?
+     * 
+     * @param node Node in question
+     * @return true iff we have no requests still being sent to the given node
+     */
+    public boolean canSendMore(String node) {
+        Deque<ClientRequest> queue = requests.get(node);
+        return queue == null || queue.isEmpty() ||
+               (queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection);
+    }
+
+    /**
+     * Return the number of inflight requests directed at the given node
+     * @param node The node
+     * @return The request count.
+     */
+    public int inFlightRequestCount(String node) {
+        Deque<ClientRequest> queue = requests.get(node);
+        return queue == null ? 0 : queue.size();
+    }
+
+    /**
+     * Count all in-flight requests for all nodes
+     */
+    public int inFlightRequestCount() {
+        int total = 0;
+        for (Deque<ClientRequest> deque : this.requests.values())
+            total += deque.size();
+        return total;
+    }
+
+    /**
+     * Clear out all the in-flight requests for the given node and return them
+     * 
+     * @param node The node
+     * @return All the in-flight requests for that node that have been removed
+     */
+    public Iterable<ClientRequest> clearAll(String node) {
+        Deque<ClientRequest> reqs = requests.get(node);
+        if (reqs == null) {
+            return Collections.emptyList();
+        } else {
+            return requests.remove(node);
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/KafkaClient.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/KafkaClient.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/KafkaClient.java
new file mode 100644
index 0000000..2c3d733
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/KafkaClient.java
@@ -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.kafka_backport.clients;
+
+import org.apache.flink.kafka_backport.common.Node;
+import org.apache.flink.kafka_backport.common.requests.RequestHeader;
+import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
+
+import java.io.Closeable;
+import java.util.List;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * The interface for {@link NetworkClient}
+ */
+public interface KafkaClient extends Closeable {
+
+    /**
+     * Check if we are currently ready to send another request to the given node but don't attempt to connect if we
+     * aren't.
+     * 
+     * @param node The node to check
+     * @param now The current timestamp
+     */
+    public boolean isReady(Node node, long now);
+
+    /**
+     * Initiate a connection to the given node (if necessary), and return true if already connected. The readiness of a
+     * node will change only when poll is invoked.
+     * 
+     * @param node The node to connect to.
+     * @param now The current time
+     * @return true iff we are ready to immediately initiate the sending of another request to the given node.
+     */
+    public boolean ready(Node node, long now);
+
+    /**
+     * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
+     * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
+     * connections.
+     * 
+     * @param node The node to check
+     * @param now The current timestamp
+     * @return The number of milliseconds to wait.
+     */
+    public long connectionDelay(Node node, long now);
+
+    /**
+     * Check if the connection of the node has failed, based on the connection state. Such connection failure are
+     * usually transient and can be resumed in the next {@link #ready(Node, long)} }
+     * call, but there are cases where transient failures needs to be caught and re-acted upon.
+     *
+     * @param node the node to check
+     * @return true iff the connection has failed and the node is disconnected
+     */
+    public boolean connectionFailed(Node node);
+
+    /**
+     * Queue up the given request for sending. Requests can only be sent on ready connections.
+     * 
+     * @param request The request
+     */
+    public void send(ClientRequest request);
+
+    /**
+     * Do actual reads and writes from sockets.
+     * 
+     * @param timeout The maximum amount of time to wait for responses in ms
+     * @param now The current time in ms
+     * @throws IllegalStateException If a request is sent to an unready node
+     */
+    public List<ClientResponse> poll(long timeout, long now);
+
+    /**
+     * Complete all in-flight requests for a given connection
+     * 
+     * @param id The connection to complete requests for
+     * @param now The current time in ms
+     * @return All requests that complete during this time period.
+     */
+    public List<ClientResponse> completeAll(String id, long now);
+
+    /**
+     * Complete all in-flight requests
+     * 
+     * @param now The current time in ms
+     * @return All requests that complete during this time period.
+     */
+    public List<ClientResponse> completeAll(long now);
+
+    /**
+     * Choose the node with the fewest outstanding requests. This method will prefer a node with an existing connection,
+     * but will potentially choose a node for which we don't yet have a connection if all existing connections are in
+     * use.
+     * 
+     * @param now The current time in ms
+     * @return The node with the fewest in-flight requests.
+     */
+    public Node leastLoadedNode(long now);
+
+    /**
+     * The number of currently in-flight requests for which we have not yet returned a response
+     */
+    public int inFlightRequestCount();
+
+    /**
+     * Get the total in-flight requests for a particular node
+     * 
+     * @param nodeId The id of the node
+     */
+    public int inFlightRequestCount(String nodeId);
+
+    /**
+     * Generate a request header for the next request
+     * 
+     * @param key The API key of the request
+     */
+    public RequestHeader nextRequestHeader(ApiKeys key);
+
+    /**
+     * Wake up the client if it is currently blocked waiting for I/O
+     */
+    public void wakeup();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/Metadata.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/Metadata.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/Metadata.java
new file mode 100644
index 0000000..1d228f7
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/Metadata.java
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.flink.kafka_backport.clients;
+
+import org.apache.flink.kafka_backport.common.Cluster;
+import org.apache.flink.kafka_backport.common.errors.TimeoutException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.Set;
+
+// ----------------------------------------------------------------------------
+//  This class is copied from the Apache Kafka project.
+// 
+//  The class is part of a "backport" of the new consumer API, in order to
+//  give Flink access to its functionality until the API is properly released.
+// 
+//  This is a temporary workaround!
+// ----------------------------------------------------------------------------
+
+/**
+ * A class encapsulating some of the logic around metadata.
+ * <p>
+ * This class is shared by the client thread (for partitioning) and the background sender thread.
+ * 
+ * Metadata is maintained for only a subset of topics, which can be added to over time. When we request metadata for a
+ * topic we don't have any metadata for it will trigger a metadata update.
+ */
+public final class Metadata {
+
+    private static final Logger log = LoggerFactory.getLogger(Metadata.class);
+
+    private final long refreshBackoffMs;
+    private final long metadataExpireMs;
+    private int version;
+    private long lastRefreshMs;
+    private long lastSuccessfulRefreshMs;
+    private Cluster cluster;
+    private boolean needUpdate;
+    private final Set<String> topics;
+
+    /**
+     * Create a metadata instance with reasonable defaults
+     */
+    public Metadata() {
+        this(100L, 60 * 60 * 1000L);
+    }
+
+    /**
+     * Create a new Metadata instance
+     * @param refreshBackoffMs The minimum amount of time that must expire between metadata refreshes to avoid busy
+     *        polling
+     * @param metadataExpireMs The maximum amount of time that metadata can be retained without refresh
+     */
+    public Metadata(long refreshBackoffMs, long metadataExpireMs) {
+        this.refreshBackoffMs = refreshBackoffMs;
+        this.metadataExpireMs = metadataExpireMs;
+        this.lastRefreshMs = 0L;
+        this.lastSuccessfulRefreshMs = 0L;
+        this.version = 0;
+        this.cluster = Cluster.empty();
+        this.needUpdate = false;
+        this.topics = new HashSet<String>();
+    }
+
+    /**
+     * Get the current cluster info without blocking
+     */
+    public synchronized Cluster fetch() {
+        return this.cluster;
+    }
+
+    /**
+     * Add the topic to maintain in the metadata
+     */
+    public synchronized void add(String topic) {
+        topics.add(topic);
+    }
+
+    /**
+     * The next time to update the cluster info is the maximum of the time the current info will expire and the time the
+     * current info can be updated (i.e. backoff time has elapsed); If an update has been request then the expiry time
+     * is now
+     */
+    public synchronized long timeToNextUpdate(long nowMs) {
+        long timeToExpire = needUpdate ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0);
+        long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs;
+        return Math.max(timeToExpire, timeToAllowUpdate);
+    }
+
+    /**
+     * Request an update of the current cluster metadata info, return the current version before the update
+     */
+    public synchronized int requestUpdate() {
+        this.needUpdate = true;
+        return this.version;
+    }
+
+    /**
+     * Wait for metadata update until the current version is larger than the last version we know of
+     */
+    public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs) throws InterruptedException {
+        if (maxWaitMs < 0) {
+            throw new IllegalArgumentException("Max time to wait for metadata updates should not be < 0 milli seconds");
+        }
+        long begin = System.currentTimeMillis();
+        long remainingWaitMs = maxWaitMs;
+        while (this.version <= lastVersion) {
+            if (remainingWaitMs != 0)
+                wait(remainingWaitMs);
+            long elapsed = System.currentTimeMillis() - begin;
+            if (elapsed >= maxWaitMs)
+                throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
+            remainingWaitMs = maxWaitMs - elapsed;
+        }
+    }
+
+    /**
+     * Add one or more topics to maintain metadata for
+     */
+    public synchronized void addTopics(String... topics) {
+        for (String topic : topics)
+            this.topics.add(topic);
+        requestUpdate();
+    }
+
+    /**
+     * Get the list of topics we are currently maintaining metadata for
+     */
+    public synchronized Set<String> topics() {
+        return new HashSet<String>(this.topics);
+    }
+
+    /**
+     * Check if a topic is already in the topic set.
+     * @param topic topic to check
+     * @return true if the topic exists, false otherwise
+     */
+    public synchronized boolean containsTopic(String topic) {
+        return this.topics.contains(topic);
+    }
+
+    /**
+     * Update the cluster metadata
+     */
+    public synchronized void update(Cluster cluster, long now) {
+        this.needUpdate = false;
+        this.lastRefreshMs = now;
+        this.lastSuccessfulRefreshMs = now;
+        this.version += 1;
+        this.cluster = cluster;
+        notifyAll();
+        log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster);
+    }
+    
+    /**
+     * Record an attempt to update the metadata that failed. We need to keep track of this
+     * to avoid retrying immediately.
+     */
+    public synchronized void failedUpdate(long now) {
+        this.lastRefreshMs = now;
+    }
+    
+    /**
+     * @return The current metadata version
+     */
+    public synchronized int version() {
+        return this.version;
+    }
+
+    /**
+     * The last time metadata was successfully updated.
+     */
+    public synchronized long lastSuccessfulUpdate() {
+        return this.lastSuccessfulRefreshMs;
+    }
+
+    /**
+     * The metadata refresh backoff in ms
+     */
+    public long refreshBackoff() {
+        return refreshBackoffMs;
+    }
+}


[50/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

Posted by se...@apache.org.
[FLINK-2386] [kafka connector] Remove copied Kafka code again. Implemented our own topic metadata retrieval.

This closes #1039


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

Branch: refs/heads/master
Commit: 76fcaca81e60edf68d2e1232556360fae62dac54
Parents: b9892a0
Author: Robert Metzger <rm...@apache.org>
Authored: Mon Aug 17 21:16:43 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Aug 27 12:40:38 2015 +0200

----------------------------------------------------------------------
 docs/_includes/navbar.html                      |    4 +-
 docs/apis/kafka.md                              |    5 +-
 .../api/FlinkLocalCluster.java                  |    6 +-
 .../java/org/apache/flink/util/NetUtils.java    |    5 +-
 .../flink-connector-kafka-083/pom.xml           |  111 --
 .../kafka_backport/clients/ClientRequest.java   |   75 --
 .../kafka_backport/clients/ClientResponse.java  |   87 --
 .../kafka_backport/clients/ClientUtils.java     |   71 -
 .../clients/ClusterConnectionStates.java        |  170 ---
 .../clients/CommonClientConfigs.java            |   69 -
 .../kafka_backport/clients/ConnectionState.java |   29 -
 .../clients/InFlightRequests.java               |  135 --
 .../kafka_backport/clients/KafkaClient.java     |  143 --
 .../flink/kafka_backport/clients/Metadata.java  |  195 ---
 .../kafka_backport/clients/NetworkClient.java   |  528 --------
 .../clients/RequestCompletionHandler.java       |   32 -
 .../clients/consumer/CommitType.java            |   26 -
 .../clients/consumer/Consumer.java              |  135 --
 .../consumer/ConsumerCommitCallback.java        |   42 -
 .../clients/consumer/ConsumerConfig.java        |  334 -----
 .../consumer/ConsumerRebalanceCallback.java     |  104 --
 .../clients/consumer/ConsumerRecord.java        |   93 --
 .../clients/consumer/ConsumerRecords.java       |  126 --
 .../consumer/ConsumerWakeupException.java       |   29 -
 .../clients/consumer/KafkaConsumer.java         | 1130 ----------------
 .../clients/consumer/MockConsumer.java          |  209 ---
 .../consumer/NoOffsetForPartitionException.java |   38 -
 .../clients/consumer/OffsetResetStrategy.java   |   26 -
 .../internals/ConsumerNetworkClient.java        |  296 -----
 .../clients/consumer/internals/Coordinator.java |  791 -----------
 .../clients/consumer/internals/DelayedTask.java |   24 -
 .../consumer/internals/DelayedTaskQueue.java    |   96 --
 .../clients/consumer/internals/Fetcher.java     |  506 --------
 .../clients/consumer/internals/Heartbeat.java   |   83 --
 .../internals/NoAvailableBrokersException.java  |   32 -
 .../NoOpConsumerRebalanceCallback.java          |   39 -
 .../consumer/internals/RequestFuture.java       |  211 ---
 .../internals/RequestFutureAdapter.java         |   37 -
 .../internals/RequestFutureListener.java        |   32 -
 .../consumer/internals/SendFailedException.java |   36 -
 .../internals/StaleMetadataException.java       |   31 -
 .../consumer/internals/SubscriptionState.java   |  242 ----
 .../flink/kafka_backport/common/Cluster.java    |  203 ---
 .../kafka_backport/common/Configurable.java     |   40 -
 .../kafka_backport/common/KafkaException.java   |   51 -
 .../flink/kafka_backport/common/Metric.java     |   43 -
 .../flink/kafka_backport/common/MetricName.java |  194 ---
 .../flink/kafka_backport/common/Node.java       |  113 --
 .../kafka_backport/common/PartitionInfo.java    |  104 --
 .../kafka_backport/common/TopicPartition.java   |   89 --
 .../common/config/AbstractConfig.java           |  185 ---
 .../kafka_backport/common/config/ConfigDef.java |  456 -------
 .../common/config/ConfigException.java          |   49 -
 .../common/errors/ApiException.java             |   60 -
 ...onsumerCoordinatorNotAvailableException.java |   49 -
 .../common/errors/CorruptRecordException.java   |   48 -
 .../common/errors/DisconnectException.java      |   47 -
 .../errors/IllegalGenerationException.java      |   42 -
 .../common/errors/InterruptException.java       |   48 -
 .../common/errors/InvalidMetadataException.java |   48 -
 .../errors/InvalidRequiredAcksException.java    |   34 -
 .../common/errors/InvalidTopicException.java    |   47 -
 .../errors/LeaderNotAvailableException.java     |   36 -
 .../common/errors/NetworkException.java         |   48 -
 .../NotCoordinatorForConsumerException.java     |   49 -
 .../NotEnoughReplicasAfterAppendException.java  |   39 -
 .../errors/NotEnoughReplicasException.java      |   49 -
 .../errors/NotLeaderForPartitionException.java  |   47 -
 .../errors/OffsetLoadInProgressException.java   |   49 -
 .../common/errors/OffsetMetadataTooLarge.java   |   46 -
 .../errors/OffsetOutOfRangeException.java       |   47 -
 .../errors/RecordBatchTooLargeException.java    |   48 -
 .../common/errors/RecordTooLargeException.java  |   47 -
 .../common/errors/RetriableException.java       |   46 -
 .../common/errors/SerializationException.java   |   55 -
 .../common/errors/TimeoutException.java         |   47 -
 .../errors/UnknownConsumerIdException.java      |   42 -
 .../common/errors/UnknownServerException.java   |   48 -
 .../UnknownTopicOrPartitionException.java       |   46 -
 .../common/metrics/CompoundStat.java            |   61 -
 .../common/metrics/JmxReporter.java             |  225 ----
 .../common/metrics/KafkaMetric.java             |   74 --
 .../common/metrics/Measurable.java              |   37 -
 .../common/metrics/MeasurableStat.java          |   38 -
 .../common/metrics/MetricConfig.java            |   96 --
 .../kafka_backport/common/metrics/Metrics.java  |  211 ---
 .../common/metrics/MetricsReporter.java         |   50 -
 .../kafka_backport/common/metrics/Quota.java    |   61 -
 .../common/metrics/QuotaViolationException.java |   41 -
 .../kafka_backport/common/metrics/Sensor.java   |  182 ---
 .../kafka_backport/common/metrics/Stat.java     |   41 -
 .../common/metrics/stats/Avg.java               |   54 -
 .../common/metrics/stats/Count.java             |   50 -
 .../common/metrics/stats/Histogram.java         |  166 ---
 .../common/metrics/stats/Max.java               |   50 -
 .../common/metrics/stats/Min.java               |   50 -
 .../common/metrics/stats/Percentile.java        |   49 -
 .../common/metrics/stats/Percentiles.java       |  125 --
 .../common/metrics/stats/Rate.java              |  115 --
 .../common/metrics/stats/SampledStat.java       |  139 --
 .../common/metrics/stats/Total.java             |   52 -
 .../common/network/ByteBufferReceive.java       |   66 -
 .../common/network/ByteBufferSend.java          |   71 -
 .../common/network/InvalidReceiveException.java |   39 -
 .../common/network/MultiSend.java               |  108 --
 .../common/network/NetworkReceive.java          |  128 --
 .../common/network/NetworkSend.java             |   51 -
 .../kafka_backport/common/network/Receive.java  |   54 -
 .../common/network/Selectable.java              |  115 --
 .../kafka_backport/common/network/Selector.java |  664 ----------
 .../kafka_backport/common/network/Send.java     |   56 -
 .../kafka_backport/common/protocol/ApiKeys.java |   75 --
 .../kafka_backport/common/protocol/Errors.java  |  172 ---
 .../common/protocol/ProtoUtils.java             |   74 --
 .../common/protocol/Protocol.java               |  474 -------
 .../common/protocol/SecurityProtocol.java       |   72 -
 .../common/protocol/types/ArrayOf.java          |   88 --
 .../common/protocol/types/Field.java            |   78 --
 .../common/protocol/types/Schema.java           |  168 ---
 .../common/protocol/types/SchemaException.java  |   41 -
 .../common/protocol/types/Struct.java           |  338 -----
 .../common/protocol/types/Type.java             |  259 ----
 .../common/record/ByteBufferInputStream.java    |   58 -
 .../common/record/ByteBufferOutputStream.java   |   66 -
 .../common/record/CompressionType.java          |   72 -
 .../common/record/Compressor.java               |  279 ----
 .../common/record/InvalidRecordException.java   |   36 -
 .../common/record/KafkaLZ4BlockInputStream.java |  236 ----
 .../record/KafkaLZ4BlockOutputStream.java       |  400 ------
 .../kafka_backport/common/record/LogEntry.java  |   57 -
 .../common/record/MemoryRecords.java            |  280 ----
 .../kafka_backport/common/record/Record.java    |  352 -----
 .../kafka_backport/common/record/Records.java   |   54 -
 .../common/requests/AbstractRequest.java        |   71 -
 .../requests/AbstractRequestResponse.java       |   75 --
 .../requests/ConsumerMetadataRequest.java       |   74 --
 .../requests/ConsumerMetadataResponse.java      |   79 --
 .../common/requests/FetchRequest.java           |  174 ---
 .../common/requests/FetchResponse.java          |  134 --
 .../common/requests/HeartbeatRequest.java       |   90 --
 .../common/requests/HeartbeatResponse.java      |   64 -
 .../common/requests/JoinGroupRequest.java       |  121 --
 .../common/requests/JoinGroupResponse.java      |  122 --
 .../common/requests/ListOffsetRequest.java      |  151 ---
 .../common/requests/ListOffsetResponse.java     |  127 --
 .../common/requests/MetadataRequest.java        |   89 --
 .../common/requests/MetadataResponse.java       |  186 ---
 .../common/requests/OffsetCommitRequest.java    |  275 ----
 .../common/requests/OffsetCommitResponse.java   |  109 --
 .../common/requests/OffsetFetchRequest.java     |  132 --
 .../common/requests/OffsetFetchResponse.java    |  135 --
 .../common/requests/ProduceRequest.java         |  141 --
 .../common/requests/ProduceResponse.java        |  131 --
 .../common/requests/RequestHeader.java          |   89 --
 .../common/requests/RequestSend.java            |   64 -
 .../common/requests/ResponseHeader.java         |   62 -
 .../common/requests/ResponseSend.java           |   50 -
 .../serialization/ByteArrayDeserializer.java    |   43 -
 .../serialization/ByteArraySerializer.java      |   43 -
 .../common/serialization/Deserializer.java      |   53 -
 .../serialization/IntegerDeserializer.java      |   53 -
 .../common/serialization/IntegerSerializer.java |   47 -
 .../common/serialization/Serializer.java        |   58 -
 .../serialization/StringDeserializer.java       |   62 -
 .../common/serialization/StringSerializer.java  |   62 -
 .../common/utils/AbstractIterator.java          |   97 --
 .../common/utils/CollectionUtils.java           |   71 -
 .../common/utils/CopyOnWriteMap.java            |  151 ---
 .../kafka_backport/common/utils/Crc32.java      |  396 ------
 .../common/utils/KafkaThread.java               |   44 -
 .../kafka_backport/common/utils/SystemTime.java |   52 -
 .../flink/kafka_backport/common/utils/Time.java |   48 -
 .../kafka_backport/common/utils/Utils.java      |  506 --------
 .../flink/kafka_backport/package-info.java      |   27 -
 .../connectors/FlinkKafkaConsumer.java          |  615 ---------
 .../connectors/FlinkKafkaConsumer081.java       |   57 -
 .../connectors/FlinkKafkaConsumer082.java       |   48 -
 .../connectors/FlinkKafkaConsumer083.java       |   48 -
 .../flink/streaming/connectors/KafkaSink.java   |  186 ---
 .../connectors/PartitionerWrapper.java          |   49 -
 .../SerializableKafkaPartitioner.java           |   25 -
 .../streaming/connectors/internals/Fetcher.java |   83 --
 .../connectors/internals/LegacyFetcher.java     |  602 ---------
 .../internals/NewConsumerApiFetcher.java        |  123 --
 .../connectors/internals/OffsetHandler.java     |   57 -
 .../internals/ZooKeeperStringSerializer.java    |   51 -
 .../internals/ZookeeperOffsetHandler.java       |  128 --
 .../streaming/connectors/Kafka081ITCase.java    |  100 --
 .../streaming/connectors/Kafka082ITCase.java    |  102 --
 .../KafkaConsumerPartitionAssignmentTest.java   |  256 ----
 .../streaming/connectors/KafkaConsumerTest.java |  143 --
 .../connectors/KafkaConsumerTestBase.java       | 1137 ----------------
 .../connectors/KafkaLocalSystemTime.java        |   48 -
 .../connectors/KafkaProducerITCase.java         |  186 ---
 .../streaming/connectors/KafkaTestBase.java     |  329 -----
 .../internals/ZookeeperOffsetHandlerTest.java   |   59 -
 .../connectors/testutils/DataGenerators.java    |  211 ---
 .../connectors/testutils/DiscardingSink.java    |   33 -
 .../testutils/FailingIdentityMapper.java        |  115 --
 .../testutils/JobManagerCommunicationUtils.java |   76 --
 .../testutils/MockRuntimeContext.java           |  131 --
 .../testutils/PartitionValidatingMapper.java    |   53 -
 .../connectors/testutils/SuccessException.java  |   26 -
 .../connectors/testutils/ThrottledMapper.java   |   44 -
 .../connectors/testutils/Tuple2Partitioner.java |   49 -
 .../testutils/ValidatingExactlyOnceSink.java    |   76 --
 .../src/test/resources/log4j-test.properties    |   29 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-kafka/pom.xml               |    8 +
 .../connectors/FlinkKafkaConsumer.java          |  681 ++++++++++
 .../connectors/FlinkKafkaConsumer081.java       |   57 +
 .../connectors/FlinkKafkaConsumer082.java       |   51 +
 .../flink/streaming/connectors/KafkaSink.java   |  186 +++
 .../connectors/PartitionerWrapper.java          |   49 +
 .../SerializableKafkaPartitioner.java           |   25 +
 .../streaming/connectors/internals/Fetcher.java |   83 ++
 .../connectors/internals/LegacyFetcher.java     |  596 +++++++++
 .../connectors/internals/OffsetHandler.java     |   58 +
 .../internals/ZooKeeperStringSerializer.java    |   51 +
 .../internals/ZookeeperOffsetHandler.java       |  128 ++
 .../connectors/kafka/KafkaConsumerExample.java  |   58 -
 .../connectors/kafka/KafkaProducerExample.java  |   67 -
 .../connectors/kafka/api/KafkaSink.java         |  172 +--
 .../connectors/kafka/api/KafkaSource.java       |  218 ----
 .../kafka/api/config/PartitionerWrapper.java    |   49 -
 .../api/persistent/PersistentKafkaSource.java   |  372 +-----
 .../partitioner/KafkaConstantPartitioner.java   |   33 -
 .../SerializableKafkaPartitioner.java           |   24 -
 .../KafkaConsumerPartitionAssignmentTest.java   |  256 ++++
 .../streaming/connectors/KafkaConsumerTest.java |  143 ++
 .../connectors/KafkaConsumerTestBase.java       | 1137 ++++++++++++++++
 .../flink/streaming/connectors/KafkaITCase.java |  100 ++
 .../connectors/KafkaLocalSystemTime.java        |   48 +
 .../connectors/KafkaProducerITCase.java         |  186 +++
 .../streaming/connectors/KafkaTestBase.java     |  320 +++++
 .../internals/ZookeeperOffsetHandlerTest.java   |   59 +
 .../streaming/connectors/kafka/KafkaITCase.java | 1225 ------------------
 .../kafka/util/KafkaLocalSystemTime.java        |   48 -
 .../connectors/testutils/DataGenerators.java    |  211 +++
 .../connectors/testutils/DiscardingSink.java    |   33 +
 .../testutils/FailingIdentityMapper.java        |  115 ++
 .../testutils/JobManagerCommunicationUtils.java |   76 ++
 .../testutils/MockRuntimeContext.java           |  131 ++
 .../testutils/PartitionValidatingMapper.java    |   53 +
 .../connectors/testutils/SuccessException.java  |   26 +
 .../connectors/testutils/ThrottledMapper.java   |   44 +
 .../connectors/testutils/Tuple2Partitioner.java |   49 +
 .../testutils/ValidatingExactlyOnceSink.java    |   81 ++
 .../src/test/resources/log4j-test.properties    |    6 +-
 .../api/environment/LocalStreamEnvironment.java |    3 +-
 .../environment/StreamExecutionEnvironment.java |    2 +-
 .../streaming/runtime/io/BarrierBuffer.java     |   45 +-
 .../flink/streaming/util/ClusterUtil.java       |   22 +-
 .../checkpointing/StateCheckpoinedITCase.java   |   14 +-
 tools/maven/suppressions.xml                    |    1 -
 255 files changed, 5099 insertions(+), 28364 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/docs/_includes/navbar.html
----------------------------------------------------------------------
diff --git a/docs/_includes/navbar.html b/docs/_includes/navbar.html
index 91a32ad..26ad740 100644
--- a/docs/_includes/navbar.html
+++ b/docs/_includes/navbar.html
@@ -81,8 +81,8 @@ under the License.
                 <li><a href="{{ apis }}/web_client.html">Web Client</a></li>
                 <li><a href="{{ apis }}/iterations.html">Iterations</a></li>
                 <li><a href="{{ apis }}/java8.html">Java 8</a></li>
-                <li><a href="{{ apis }}/hadoop_compatibility.html">Hadoop Compatability <span class="badge">Beta</span></a></li>
-                <li><a href="{{ apis }}/storm_compatibility.html">Storm Compatability <span class="badge">Beta</span></a></li>
+                <li><a href="{{ apis }}/hadoop_compatibility.html">Hadoop Compatibility <span class="badge">Beta</span></a></li>
+                <li><a href="{{ apis }}/storm_compatibility.html">Storm Compatibility <span class="badge">Beta</span></a></li>
               </ul>
             </li>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/docs/apis/kafka.md
----------------------------------------------------------------------
diff --git a/docs/apis/kafka.md b/docs/apis/kafka.md
index d629331..0c0790a 100644
--- a/docs/apis/kafka.md
+++ b/docs/apis/kafka.md
@@ -57,8 +57,7 @@ Please pick a package (maven artifact id) and class name for your use-case and e
 | -------------               |-------------| -----| ------ | ------ |
 | flink-connector-kafka       | 0.9, 0.10 | `KafkaSource` | 0.8.1, 0.8.2 | **No**, does not participate in checkpointing at all. | Uses the old, high level KafkaConsumer API, autocommits to ZK by Kafka |
 | flink-connector-kafka       | 0.9, 0.10 | `PersistentKafkaSource` | 0.8.1, 0.8.2 | **No**, does not guarantee exactly-once processing, element order or strict partition assignment | Uses the old, high level KafkaConsumer API, offsets are committed into ZK manually |
-| flink-connector-kafka-083   | 0.10      | `FlinkKafkaConsumer081` | 0.8.1  | **yes** | Uses the [SimpleConsumer](https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example) API of Kafka internally. Offsets are committed to ZK manually |
-| flink-connector-kafka-083   | 0.10      | `FlinkKafkaConsumer082` | 0.8.2  | **yes** | Uses the new, unreleased consumer API of Kafka 0.9.3 internally. Offsets are committed to ZK manually |
-| flink-connector-kafka-083   | 0.10      | `FlinkKafkaConsumer083` | 0.8.3  | **yes** | **EXPERIMENTAL** Uses the new, unreleased consumer of Kafka 0.9.3. Offsets are committed using the Consumer API |
+| flink-connector-kafka-083   | 0.9.1 0.10 | `FlinkKafkaConsumer081` | 0.8.1  | **yes** | Uses the [SimpleConsumer](https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example) API of Kafka internally. Offsets are committed to ZK manually |
+| flink-connector-kafka-083   | 0.9.1 0.10 | `FlinkKafkaConsumer082` | 0.8.2  | **yes** | Uses the [SimpleConsumer](https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example) API of Kafka internally. Offsets are committed to ZK manually |
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
index 880e504..b5eda8b 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
@@ -41,7 +41,7 @@ public class FlinkLocalCluster {
 
 	public void submitTopologyWithOpts(final String topologyName, final Map<?, ?> conf, final FlinkTopology topology,
 			final SubmitOptions submitOpts) throws Exception {
-		ClusterUtil.startOnMiniCluster(topology.getStreamGraph().getJobGraph(topologyName), topology.getNumberOfTasks(), -1);
+		ClusterUtil.startOnMiniCluster(topology.getStreamGraph().getJobGraph(topologyName), topology.getNumberOfTasks());
 	}
 
 	public void killTopology(final String topologyName) {
@@ -60,9 +60,7 @@ public class FlinkLocalCluster {
 	public void rebalance(final String name, final RebalanceOptions options) {
 	}
 
-	public void shutdown() {
-		ClusterUtil.stopOnMiniCluster();
-	}
+	public void shutdown() {}
 
 	public String getTopologyConf(final String id) {
 		return null;

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-core/src/main/java/org/apache/flink/util/NetUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/NetUtils.java b/flink-core/src/main/java/org/apache/flink/util/NetUtils.java
index 070a650..abbcf3c 100644
--- a/flink-core/src/main/java/org/apache/flink/util/NetUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/NetUtils.java
@@ -48,8 +48,10 @@ public class NetUtils {
 	 * Works also for ipv6.
 	 *
 	 * See: http://stackoverflow.com/questions/2345063/java-common-way-to-validate-and-convert-hostport-to-inetsocketaddress
+	 *
+	 * @return URL object for accessing host and Port
 	 */
-	public static void ensureCorrectHostnamePort(String hostPort) {
+	public static URL getCorrectHostnamePort(String hostPort) {
 		try {
 			URL u = new URL("http://"+hostPort);
 			if(u.getHost() == null) {
@@ -58,6 +60,7 @@ public class NetUtils {
 			if(u.getPort() == -1) {
 				throw new IllegalArgumentException("The given host:port ('"+hostPort+"') doesn't contain a valid port");
 			}
+			return u;
 		} catch (MalformedURLException e) {
 			throw new IllegalArgumentException("The given host:port ('"+hostPort+"') is invalid", e);
 		}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientRequest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientRequest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientRequest.java
deleted file mode 100644
index d86ea96..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientRequest.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.kafka_backport.clients;
-
-import org.apache.flink.kafka_backport.common.requests.RequestSend;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A request being sent to the server. This holds both the network send as well as the client-level metadata.
- */
-public final class ClientRequest {
-
-    private final long createdMs;
-    private final boolean expectResponse;
-    private final RequestSend request;
-    private final RequestCompletionHandler callback;
-
-    /**
-     * @param createdMs The unix timestamp in milliseconds for the time at which this request was created.
-     * @param expectResponse Should we expect a response message or is this request complete once it is sent?
-     * @param request The request
-     * @param callback A callback to execute when the response has been received (or null if no callback is necessary)
-     */
-    public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, RequestCompletionHandler callback) {
-        this.createdMs = createdMs;
-        this.callback = callback;
-        this.request = request;
-        this.expectResponse = expectResponse;
-    }
-
-    @Override
-    public String toString() {
-        return "ClientRequest(expectResponse=" + expectResponse + ", callback=" + callback + ", request=" + request
-                + ")";
-    }
-
-    public boolean expectResponse() {
-        return expectResponse;
-    }
-
-    public RequestSend request() {
-        return request;
-    }
-
-    public boolean hasCallback() {
-        return callback != null;
-    }
-
-    public RequestCompletionHandler callback() {
-        return callback;
-    }
-
-    public long createdTime() {
-        return createdMs;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientResponse.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientResponse.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientResponse.java
deleted file mode 100644
index 49a7540..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientResponse.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients;
-
-import org.apache.flink.kafka_backport.common.protocol.types.Struct;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A response from the server. Contains both the body of the response as well as the correlated request that was
- * originally sent.
- */
-public class ClientResponse {
-
-    private final long received;
-    private final boolean disconnected;
-    private final ClientRequest request;
-    private final Struct responseBody;
-
-    /**
-     * @param request The original request
-     * @param received The unix timestamp when this response was received
-     * @param disconnected Whether the client disconnected before fully reading a response
-     * @param responseBody The response contents (or null) if we disconnected or no response was expected
-     */
-    public ClientResponse(ClientRequest request, long received, boolean disconnected, Struct responseBody) {
-        super();
-        this.received = received;
-        this.disconnected = disconnected;
-        this.request = request;
-        this.responseBody = responseBody;
-    }
-
-    public long receivedTime() {
-        return received;
-    }
-
-    public boolean wasDisconnected() {
-        return disconnected;
-    }
-
-    public ClientRequest request() {
-        return request;
-    }
-
-    public Struct responseBody() {
-        return responseBody;
-    }
-
-    public boolean hasResponse() {
-        return responseBody != null;
-    }
-
-    public long requestLatencyMs() {
-        return receivedTime() - this.request.createdTime();
-    }
-
-    @Override
-    public String toString() {
-        return "ClientResponse(received=" + received +
-               ", disconnected=" +
-               disconnected +
-               ", request=" +
-               request +
-               ", responseBody=" +
-               responseBody +
-               ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientUtils.java
deleted file mode 100644
index 1a00a78..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClientUtils.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.kafka_backport.clients;
-
-import org.apache.flink.kafka_backport.common.config.ConfigException;
-import org.apache.flink.kafka_backport.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicReference;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-public class ClientUtils {
-    private static final Logger log = LoggerFactory.getLogger(ClientUtils.class);
-
-    public static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls) {
-        List<InetSocketAddress> addresses = new ArrayList<InetSocketAddress>();
-        for (String url : urls) {
-            if (url != null && url.length() > 0) {
-                String host = Utils.getHost(url);
-                Integer port = Utils.getPort(url);
-                if (host == null || port == null)
-                    throw new ConfigException("Invalid url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
-                try {
-                    InetSocketAddress address = new InetSocketAddress(host, port);
-                    if (address.isUnresolved())
-                        throw new ConfigException("DNS resolution failed for url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
-                    addresses.add(address);
-                } catch (NumberFormatException e) {
-                    throw new ConfigException("Invalid port in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
-                }
-            }
-        }
-        if (addresses.size() < 1)
-            throw new ConfigException("No bootstrap urls given in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
-        return addresses;
-    }
-
-    public static void closeQuietly(Closeable c, String name, AtomicReference<Throwable> firstException) {
-        if (c != null) {
-            try {
-                c.close();
-            } catch (Throwable t) {
-                firstException.compareAndSet(null, t);
-                log.error("Failed to close " + name, t);
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClusterConnectionStates.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClusterConnectionStates.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClusterConnectionStates.java
deleted file mode 100644
index 395164b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ClusterConnectionStates.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients;
-
-import java.util.HashMap;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The state of our connection to each node in the cluster.
- * 
- */
-final class ClusterConnectionStates {
-    private final long reconnectBackoffMs;
-    private final Map<String, NodeConnectionState> nodeState;
-
-    public ClusterConnectionStates(long reconnectBackoffMs) {
-        this.reconnectBackoffMs = reconnectBackoffMs;
-        this.nodeState = new HashMap<String, NodeConnectionState>();
-    }
-
-    /**
-     * Return true iff we can currently initiate a new connection. This will be the case if we are not
-     * connected and haven't been connected for at least the minimum reconnection backoff period.
-     * @param id The connection id to check
-     * @param now The current time in MS
-     * @return true if we can initiate a new connection
-     */
-    public boolean canConnect(String id, long now) {
-        NodeConnectionState state = nodeState.get(id);
-        if (state == null)
-            return true;
-        else
-            return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs >= this.reconnectBackoffMs;
-    }
-
-    /**
-     * Return true if we are disconnected from the given node and can't re-establish a connection yet
-     * @param id The connection to check
-     * @param now The current time in ms
-     */
-    public boolean isBlackedOut(String id, long now) {
-        NodeConnectionState state = nodeState.get(id);
-        if (state == null)
-            return false;
-        else
-            return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs < this.reconnectBackoffMs;
-    }
-
-    /**
-     * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
-     * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
-     * connections.
-     * @param id The connection to check
-     * @param now The current time in ms
-     */
-    public long connectionDelay(String id, long now) {
-        NodeConnectionState state = nodeState.get(id);
-        if (state == null) return 0;
-        long timeWaited = now - state.lastConnectAttemptMs;
-        if (state.state == ConnectionState.DISCONNECTED) {
-            return Math.max(this.reconnectBackoffMs - timeWaited, 0);
-        } else {
-            // When connecting or connected, we should be able to delay indefinitely since other events (connection or
-            // data acked) will cause a wakeup once data can be sent.
-            return Long.MAX_VALUE;
-        }
-    }
-
-    /**
-     * Enter the connecting state for the given connection.
-     * @param id The id of the connection
-     * @param now The current time.
-     */
-    public void connecting(String id, long now) {
-        nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now));
-    }
-
-    /**
-     * Return true iff a specific connection is connected
-     * @param id The id of the connection to check
-     */
-    public boolean isConnected(String id) {
-        NodeConnectionState state = nodeState.get(id);
-        return state != null && state.state == ConnectionState.CONNECTED;
-    }
-
-    /**
-     * Return true iff we are in the process of connecting
-     * @param id The id of the connection
-     */
-    public boolean isConnecting(String id) {
-        NodeConnectionState state = nodeState.get(id);
-        return state != null && state.state == ConnectionState.CONNECTING;
-    }
-
-    /**
-     * Enter the connected state for the given connection
-     * @param id The connection identifier
-     */
-    public void connected(String id) {
-        NodeConnectionState nodeState = nodeState(id);
-        nodeState.state = ConnectionState.CONNECTED;
-    }
-
-    /**
-     * Enter the disconnected state for the given node
-     * @param id The connection we have disconnected
-     */
-    public void disconnected(String id) {
-        NodeConnectionState nodeState = nodeState(id);
-        nodeState.state = ConnectionState.DISCONNECTED;
-    }
-    
-    /**
-     * Get the state of a given connection
-     * @param id The id of the connection
-     * @return The state of our connection
-     */
-    public ConnectionState connectionState(String id) {
-        return nodeState(id).state;
-    }
-    
-    /**
-     * Get the state of a given node
-     * @param id The connection to fetch the state for
-     */
-    private NodeConnectionState nodeState(String id) {
-        NodeConnectionState state = this.nodeState.get(id);
-        if (state == null)
-            throw new IllegalStateException("No entry found for connection " + id);
-        return state;
-    }
-    
-    /**
-     * The state of our connection to a node
-     */
-    private static class NodeConnectionState {
-
-        ConnectionState state;
-        long lastConnectAttemptMs;
-
-        public NodeConnectionState(ConnectionState state, long lastConnectAttempt) {
-            this.state = state;
-            this.lastConnectAttemptMs = lastConnectAttempt;
-        }
-
-        public String toString() {
-            return "NodeState(" + state + ", " + lastConnectAttemptMs + ")";
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/CommonClientConfigs.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/CommonClientConfigs.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/CommonClientConfigs.java
deleted file mode 100644
index 01e9da2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/CommonClientConfigs.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.kafka_backport.clients;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * Some configurations shared by both producer and consumer
- */
-public class CommonClientConfigs {
-    
-    /*
-     * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
-     */
-
-    public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
-    public static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping&mdash;this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form "
-                                                       + "<code>host1:port1,host2:port2,...</code>. Since these servers are just used for the initial connection to "
-                                                       + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of "
-                                                       + "servers (you may want more than one, though, in case a server is down).";
-    
-    public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms";
-    public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions.";
-    
-    public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes";
-    public static final String SEND_BUFFER_DOC = "The size of the TCP send buffer (SO_SNDBUF) to use when sending data.";
-
-    public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes";
-    public static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer (SO_RCVBUF) to use when reading data.";
-
-    public static final String CLIENT_ID_CONFIG = "client.id";
-    public static final String CLIENT_ID_DOC = "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging.";
-
-    public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms";
-    public static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all requests sent by the consumer to the broker.";
-
-    public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms";
-    public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop.";
-    
-    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms";
-    public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The number of samples maintained to compute metrics.";
-
-    public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples";
-    public static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics.";
-
-    public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters";
-    public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics.";
-
-    public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = "connections.max.idle.ms";
-    public static final String CONNECTIONS_MAX_IDLE_MS_DOC = "Close idle connections after the number of milliseconds specified by this config.";
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ConnectionState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ConnectionState.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ConnectionState.java
deleted file mode 100644
index 34fd7e2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/ConnectionState.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.kafka_backport.clients;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The states of a node connection
- */
-public enum ConnectionState {
-    DISCONNECTED, CONNECTING, CONNECTED
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/InFlightRequests.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/InFlightRequests.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/InFlightRequests.java
deleted file mode 100644
index 9fc489b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/InFlightRequests.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.kafka_backport.clients;
-
-import java.util.ArrayDeque;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.Map;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The set of requests which have been sent or are being sent but haven't yet received a response
- */
-final class InFlightRequests {
-
-    private final int maxInFlightRequestsPerConnection;
-    private final Map<String, Deque<ClientRequest>> requests = new HashMap<String, Deque<ClientRequest>>();
-
-    public InFlightRequests(int maxInFlightRequestsPerConnection) {
-        this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection;
-    }
-
-    /**
-     * Add the given request to the queue for the connection it was directed to
-     */
-    public void add(ClientRequest request) {
-        Deque<ClientRequest> reqs = this.requests.get(request.request().destination());
-        if (reqs == null) {
-            reqs = new ArrayDeque<ClientRequest>();
-            this.requests.put(request.request().destination(), reqs);
-        }
-        reqs.addFirst(request);
-    }
-
-    /**
-     * Get the request queue for the given node
-     */
-    private Deque<ClientRequest> requestQueue(String node) {
-        Deque<ClientRequest> reqs = requests.get(node);
-        if (reqs == null || reqs.isEmpty())
-            throw new IllegalStateException("Response from server for which there are no in-flight requests.");
-        return reqs;
-    }
-
-    /**
-     * Get the oldest request (the one that that will be completed next) for the given node
-     */
-    public ClientRequest completeNext(String node) {
-        return requestQueue(node).pollLast();
-    }
-
-    /**
-     * Get the last request we sent to the given node (but don't remove it from the queue)
-     * @param node The node id
-     */
-    public ClientRequest lastSent(String node) {
-        return requestQueue(node).peekFirst();
-    }
-
-    /**
-     * Complete the last request that was sent to a particular node.
-     * @param node The node the request was sent to
-     * @return The request
-     */
-    public ClientRequest completeLastSent(String node) {
-        return requestQueue(node).pollFirst();
-    }
-
-    /**
-     * Can we send more requests to this node?
-     * 
-     * @param node Node in question
-     * @return true iff we have no requests still being sent to the given node
-     */
-    public boolean canSendMore(String node) {
-        Deque<ClientRequest> queue = requests.get(node);
-        return queue == null || queue.isEmpty() ||
-               (queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection);
-    }
-
-    /**
-     * Return the number of inflight requests directed at the given node
-     * @param node The node
-     * @return The request count.
-     */
-    public int inFlightRequestCount(String node) {
-        Deque<ClientRequest> queue = requests.get(node);
-        return queue == null ? 0 : queue.size();
-    }
-
-    /**
-     * Count all in-flight requests for all nodes
-     */
-    public int inFlightRequestCount() {
-        int total = 0;
-        for (Deque<ClientRequest> deque : this.requests.values())
-            total += deque.size();
-        return total;
-    }
-
-    /**
-     * Clear out all the in-flight requests for the given node and return them
-     * 
-     * @param node The node
-     * @return All the in-flight requests for that node that have been removed
-     */
-    public Iterable<ClientRequest> clearAll(String node) {
-        Deque<ClientRequest> reqs = requests.get(node);
-        if (reqs == null) {
-            return Collections.emptyList();
-        } else {
-            return requests.remove(node);
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/KafkaClient.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/KafkaClient.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/KafkaClient.java
deleted file mode 100644
index 2c3d733..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/KafkaClient.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients;
-
-import org.apache.flink.kafka_backport.common.Node;
-import org.apache.flink.kafka_backport.common.requests.RequestHeader;
-import org.apache.flink.kafka_backport.common.protocol.ApiKeys;
-
-import java.io.Closeable;
-import java.util.List;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * The interface for {@link NetworkClient}
- */
-public interface KafkaClient extends Closeable {
-
-    /**
-     * Check if we are currently ready to send another request to the given node but don't attempt to connect if we
-     * aren't.
-     * 
-     * @param node The node to check
-     * @param now The current timestamp
-     */
-    public boolean isReady(Node node, long now);
-
-    /**
-     * Initiate a connection to the given node (if necessary), and return true if already connected. The readiness of a
-     * node will change only when poll is invoked.
-     * 
-     * @param node The node to connect to.
-     * @param now The current time
-     * @return true iff we are ready to immediately initiate the sending of another request to the given node.
-     */
-    public boolean ready(Node node, long now);
-
-    /**
-     * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
-     * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
-     * connections.
-     * 
-     * @param node The node to check
-     * @param now The current timestamp
-     * @return The number of milliseconds to wait.
-     */
-    public long connectionDelay(Node node, long now);
-
-    /**
-     * Check if the connection of the node has failed, based on the connection state. Such connection failure are
-     * usually transient and can be resumed in the next {@link #ready(Node, long)} }
-     * call, but there are cases where transient failures needs to be caught and re-acted upon.
-     *
-     * @param node the node to check
-     * @return true iff the connection has failed and the node is disconnected
-     */
-    public boolean connectionFailed(Node node);
-
-    /**
-     * Queue up the given request for sending. Requests can only be sent on ready connections.
-     * 
-     * @param request The request
-     */
-    public void send(ClientRequest request);
-
-    /**
-     * Do actual reads and writes from sockets.
-     * 
-     * @param timeout The maximum amount of time to wait for responses in ms
-     * @param now The current time in ms
-     * @throws IllegalStateException If a request is sent to an unready node
-     */
-    public List<ClientResponse> poll(long timeout, long now);
-
-    /**
-     * Complete all in-flight requests for a given connection
-     * 
-     * @param id The connection to complete requests for
-     * @param now The current time in ms
-     * @return All requests that complete during this time period.
-     */
-    public List<ClientResponse> completeAll(String id, long now);
-
-    /**
-     * Complete all in-flight requests
-     * 
-     * @param now The current time in ms
-     * @return All requests that complete during this time period.
-     */
-    public List<ClientResponse> completeAll(long now);
-
-    /**
-     * Choose the node with the fewest outstanding requests. This method will prefer a node with an existing connection,
-     * but will potentially choose a node for which we don't yet have a connection if all existing connections are in
-     * use.
-     * 
-     * @param now The current time in ms
-     * @return The node with the fewest in-flight requests.
-     */
-    public Node leastLoadedNode(long now);
-
-    /**
-     * The number of currently in-flight requests for which we have not yet returned a response
-     */
-    public int inFlightRequestCount();
-
-    /**
-     * Get the total in-flight requests for a particular node
-     * 
-     * @param nodeId The id of the node
-     */
-    public int inFlightRequestCount(String nodeId);
-
-    /**
-     * Generate a request header for the next request
-     * 
-     * @param key The API key of the request
-     */
-    public RequestHeader nextRequestHeader(ApiKeys key);
-
-    /**
-     * Wake up the client if it is currently blocked waiting for I/O
-     */
-    public void wakeup();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/76fcaca8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/Metadata.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/Metadata.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/Metadata.java
deleted file mode 100644
index 1d228f7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/flink/kafka_backport/clients/Metadata.java
+++ /dev/null
@@ -1,195 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.flink.kafka_backport.clients;
-
-import org.apache.flink.kafka_backport.common.Cluster;
-import org.apache.flink.kafka_backport.common.errors.TimeoutException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashSet;
-import java.util.Set;
-
-// ----------------------------------------------------------------------------
-//  This class is copied from the Apache Kafka project.
-// 
-//  The class is part of a "backport" of the new consumer API, in order to
-//  give Flink access to its functionality until the API is properly released.
-// 
-//  This is a temporary workaround!
-// ----------------------------------------------------------------------------
-
-/**
- * A class encapsulating some of the logic around metadata.
- * <p>
- * This class is shared by the client thread (for partitioning) and the background sender thread.
- * 
- * Metadata is maintained for only a subset of topics, which can be added to over time. When we request metadata for a
- * topic we don't have any metadata for it will trigger a metadata update.
- */
-public final class Metadata {
-
-    private static final Logger log = LoggerFactory.getLogger(Metadata.class);
-
-    private final long refreshBackoffMs;
-    private final long metadataExpireMs;
-    private int version;
-    private long lastRefreshMs;
-    private long lastSuccessfulRefreshMs;
-    private Cluster cluster;
-    private boolean needUpdate;
-    private final Set<String> topics;
-
-    /**
-     * Create a metadata instance with reasonable defaults
-     */
-    public Metadata() {
-        this(100L, 60 * 60 * 1000L);
-    }
-
-    /**
-     * Create a new Metadata instance
-     * @param refreshBackoffMs The minimum amount of time that must expire between metadata refreshes to avoid busy
-     *        polling
-     * @param metadataExpireMs The maximum amount of time that metadata can be retained without refresh
-     */
-    public Metadata(long refreshBackoffMs, long metadataExpireMs) {
-        this.refreshBackoffMs = refreshBackoffMs;
-        this.metadataExpireMs = metadataExpireMs;
-        this.lastRefreshMs = 0L;
-        this.lastSuccessfulRefreshMs = 0L;
-        this.version = 0;
-        this.cluster = Cluster.empty();
-        this.needUpdate = false;
-        this.topics = new HashSet<String>();
-    }
-
-    /**
-     * Get the current cluster info without blocking
-     */
-    public synchronized Cluster fetch() {
-        return this.cluster;
-    }
-
-    /**
-     * Add the topic to maintain in the metadata
-     */
-    public synchronized void add(String topic) {
-        topics.add(topic);
-    }
-
-    /**
-     * The next time to update the cluster info is the maximum of the time the current info will expire and the time the
-     * current info can be updated (i.e. backoff time has elapsed); If an update has been request then the expiry time
-     * is now
-     */
-    public synchronized long timeToNextUpdate(long nowMs) {
-        long timeToExpire = needUpdate ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0);
-        long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs;
-        return Math.max(timeToExpire, timeToAllowUpdate);
-    }
-
-    /**
-     * Request an update of the current cluster metadata info, return the current version before the update
-     */
-    public synchronized int requestUpdate() {
-        this.needUpdate = true;
-        return this.version;
-    }
-
-    /**
-     * Wait for metadata update until the current version is larger than the last version we know of
-     */
-    public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs) throws InterruptedException {
-        if (maxWaitMs < 0) {
-            throw new IllegalArgumentException("Max time to wait for metadata updates should not be < 0 milli seconds");
-        }
-        long begin = System.currentTimeMillis();
-        long remainingWaitMs = maxWaitMs;
-        while (this.version <= lastVersion) {
-            if (remainingWaitMs != 0)
-                wait(remainingWaitMs);
-            long elapsed = System.currentTimeMillis() - begin;
-            if (elapsed >= maxWaitMs)
-                throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
-            remainingWaitMs = maxWaitMs - elapsed;
-        }
-    }
-
-    /**
-     * Add one or more topics to maintain metadata for
-     */
-    public synchronized void addTopics(String... topics) {
-        for (String topic : topics)
-            this.topics.add(topic);
-        requestUpdate();
-    }
-
-    /**
-     * Get the list of topics we are currently maintaining metadata for
-     */
-    public synchronized Set<String> topics() {
-        return new HashSet<String>(this.topics);
-    }
-
-    /**
-     * Check if a topic is already in the topic set.
-     * @param topic topic to check
-     * @return true if the topic exists, false otherwise
-     */
-    public synchronized boolean containsTopic(String topic) {
-        return this.topics.contains(topic);
-    }
-
-    /**
-     * Update the cluster metadata
-     */
-    public synchronized void update(Cluster cluster, long now) {
-        this.needUpdate = false;
-        this.lastRefreshMs = now;
-        this.lastSuccessfulRefreshMs = now;
-        this.version += 1;
-        this.cluster = cluster;
-        notifyAll();
-        log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster);
-    }
-    
-    /**
-     * Record an attempt to update the metadata that failed. We need to keep track of this
-     * to avoid retrying immediately.
-     */
-    public synchronized void failedUpdate(long now) {
-        this.lastRefreshMs = now;
-    }
-    
-    /**
-     * @return The current metadata version
-     */
-    public synchronized int version() {
-        return this.version;
-    }
-
-    /**
-     * The last time metadata was successfully updated.
-     */
-    public synchronized long lastSuccessfulUpdate() {
-        return this.lastSuccessfulRefreshMs;
-    }
-
-    /**
-     * The metadata refresh backoff in ms
-     */
-    public long refreshBackoff() {
-        return refreshBackoffMs;
-    }
-}


[04/51] [abbrv] flink git commit: [FLINK-2386] [kafka connector] Add comments to all backported kafka sources and move them to 'org.apache.flink.kafka_backport'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/SecurityProtocol.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/SecurityProtocol.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/SecurityProtocol.java
deleted file mode 100644
index 9a68dbb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/SecurityProtocol.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.protocol;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public enum SecurityProtocol {
-    /** Un-authenticated, non-encrypted channel */
-    PLAINTEXT(0, "PLAINTEXT"),
-    /** Currently identical to PLAINTEXT and used for testing only. We may implement extra instrumentation when testing channel code. */
-    TRACE(Short.MAX_VALUE, "TRACE");
-
-    private static final Map<Short, SecurityProtocol> CODE_TO_SECURITY_PROTOCOL = new HashMap<Short, SecurityProtocol>();
-    private static final List<String> NAMES = new ArrayList<String>();
-
-    static {
-        for (SecurityProtocol proto: SecurityProtocol.values()) {
-            CODE_TO_SECURITY_PROTOCOL.put(proto.id, proto);
-            NAMES.add(proto.name);
-        }
-    }
-
-    /** The permanent and immutable id of a security protocol -- this can't change, and must match kafka.cluster.SecurityProtocol  */
-    public final short id;
-
-    /** Name of the security protocol. This may be used by client configuration. */
-    public final String name;
-
-    private SecurityProtocol(int id, String name) {
-        this.id = (short) id;
-        this.name = name;
-    }
-
-    public static String getName(int id) {
-        return CODE_TO_SECURITY_PROTOCOL.get((short) id).name;
-    }
-
-    public static List<String> getNames() {
-        return NAMES;
-    }
-
-    public static SecurityProtocol forId(Short id) {
-        return CODE_TO_SECURITY_PROTOCOL.get(id);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/ArrayOf.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/ArrayOf.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/ArrayOf.java
deleted file mode 100644
index c7a60fe..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/ArrayOf.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.protocol.types;
-
-import java.nio.ByteBuffer;
-
-/**
- * Represents a type for an array of a particular type
- */
-public class ArrayOf extends Type {
-
-    private final Type type;
-
-    public ArrayOf(Type type) {
-        this.type = type;
-    }
-
-    @Override
-    public void write(ByteBuffer buffer, Object o) {
-        Object[] objs = (Object[]) o;
-        int size = objs.length;
-        buffer.putInt(size);
-        for (int i = 0; i < size; i++)
-            type.write(buffer, objs[i]);
-    }
-
-    @Override
-    public Object read(ByteBuffer buffer) {
-        int size = buffer.getInt();
-        Object[] objs = new Object[size];
-        for (int i = 0; i < size; i++)
-            objs[i] = type.read(buffer);
-        return objs;
-    }
-
-    @Override
-    public int sizeOf(Object o) {
-        Object[] objs = (Object[]) o;
-        int size = 4;
-        for (int i = 0; i < objs.length; i++)
-            size += type.sizeOf(objs[i]);
-        return size;
-    }
-
-    public Type type() {
-        return type;
-    }
-
-    @Override
-    public String toString() {
-        return "ARRAY(" + type + ")";
-    }
-
-    @Override
-    public Object[] validate(Object item) {
-        try {
-            Object[] array = (Object[]) item;
-            for (int i = 0; i < array.length; i++)
-                type.validate(array[i]);
-            return array;
-        } catch (ClassCastException e) {
-            throw new SchemaException("Not an Object[].");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Field.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Field.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Field.java
deleted file mode 100644
index 006e520..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Field.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.kafka.copied.common.protocol.types;
-
-/**
- * A field in a schema
- */
-public class Field {
-
-    public static final Object NO_DEFAULT = new Object();
-
-    final int index;
-    public final String name;
-    public final Type type;
-    public final Object defaultValue;
-    public final String doc;
-    final Schema schema;
-
-    /**
-     * Create the field.
-     *
-     * @throws SchemaException If the default value is not primitive and the validation fails
-     */
-    public Field(int index, String name, Type type, String doc, Object defaultValue, Schema schema) {
-        this.index = index;
-        this.name = name;
-        this.type = type;
-        this.doc = doc;
-        this.defaultValue = defaultValue;
-        this.schema = schema;
-        if (defaultValue != NO_DEFAULT)
-            type.validate(defaultValue);
-    }
-
-    public Field(int index, String name, Type type, String doc, Object defaultValue) {
-        this(index, name, type, doc, defaultValue, null);
-    }
-
-    public Field(String name, Type type, String doc, Object defaultValue) {
-        this(-1, name, type, doc, defaultValue);
-    }
-
-    public Field(String name, Type type, String doc) {
-        this(name, type, doc, NO_DEFAULT);
-    }
-
-    public Field(String name, Type type) {
-        this(name, type, "");
-    }
-
-    public Type type() {
-        return type;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Schema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Schema.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Schema.java
deleted file mode 100644
index d853e1f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Schema.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.kafka.copied.common.protocol.types;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The schema for a compound record definition
- */
-public class Schema extends Type {
-
-    private final Field[] fields;
-    private final Map<String, Field> fieldsByName;
-
-    /**
-     * Construct the schema with a given list of its field values
-     *
-     * @throws SchemaException If the given list have duplicate fields
-     */
-    public Schema(Field... fs) {
-        this.fields = new Field[fs.length];
-        this.fieldsByName = new HashMap<String, Field>();
-        for (int i = 0; i < this.fields.length; i++) {
-            Field field = fs[i];
-            if (fieldsByName.containsKey(field.name))
-                throw new SchemaException("Schema contains a duplicate field: " + field.name);
-            this.fields[i] = new Field(i, field.name, field.type, field.doc, field.defaultValue, this);
-            this.fieldsByName.put(fs[i].name, this.fields[i]);
-        }
-    }
-
-    /**
-     * Write a struct to the buffer
-     */
-    public void write(ByteBuffer buffer, Object o) {
-        Struct r = (Struct) o;
-        for (int i = 0; i < fields.length; i++) {
-            Field f = fields[i];
-            try {
-                Object value = f.type().validate(r.get(f));
-                f.type.write(buffer, value);
-            } catch (Exception e) {
-                throw new SchemaException("Error writing field '" + f.name +
-                                          "': " +
-                                          (e.getMessage() == null ? e.getClass().getName() : e.getMessage()));
-            }
-        }
-    }
-
-    /**
-     * Read a struct from the buffer
-     */
-    public Object read(ByteBuffer buffer) {
-        Object[] objects = new Object[fields.length];
-        for (int i = 0; i < fields.length; i++) {
-            try {
-                objects[i] = fields[i].type.read(buffer);
-            } catch (Exception e) {
-                throw new SchemaException("Error reading field '" + fields[i].name +
-                                          "': " +
-                                          (e.getMessage() == null ? e.getClass().getName() : e.getMessage()));
-            }
-        }
-        return new Struct(this, objects);
-    }
-
-    /**
-     * The size of the given record
-     */
-    public int sizeOf(Object o) {
-        int size = 0;
-        Struct r = (Struct) o;
-        for (int i = 0; i < fields.length; i++)
-            size += fields[i].type.sizeOf(r.get(fields[i]));
-        return size;
-    }
-
-    /**
-     * The number of fields in this schema
-     */
-    public int numFields() {
-        return this.fields.length;
-    }
-
-    /**
-     * Get a field by its slot in the record array
-     * 
-     * @param slot The slot at which this field sits
-     * @return The field
-     */
-    public Field get(int slot) {
-        return this.fields[slot];
-    }
-
-    /**
-     * Get a field by its name
-     * 
-     * @param name The name of the field
-     * @return The field
-     */
-    public Field get(String name) {
-        return this.fieldsByName.get(name);
-    }
-
-    /**
-     * Get all the fields in this schema
-     */
-    public Field[] fields() {
-        return this.fields;
-    }
-
-    /**
-     * Display a string representation of the schema
-     */
-    public String toString() {
-        StringBuilder b = new StringBuilder();
-        b.append('{');
-        for (int i = 0; i < this.fields.length; i++) {
-            b.append(this.fields[i].name);
-            b.append(':');
-            b.append(this.fields[i].type());
-            if (i < this.fields.length - 1)
-                b.append(',');
-        }
-        b.append("}");
-        return b.toString();
-    }
-
-    @Override
-    public Struct validate(Object item) {
-        try {
-            Struct struct = (Struct) item;
-            for (int i = 0; i < this.fields.length; i++) {
-                Field field = this.fields[i];
-                try {
-                    field.type.validate(struct.get(field));
-                } catch (SchemaException e) {
-                    throw new SchemaException("Invalid value for field '" + field.name + "': " + e.getMessage());
-                }
-            }
-            return struct;
-        } catch (ClassCastException e) {
-            throw new SchemaException("Not a Struct.");
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/SchemaException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/SchemaException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/SchemaException.java
deleted file mode 100644
index 4e2cb27..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/SchemaException.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.kafka.copied.common.protocol.types;
-
-import org.apache.kafka.copied.common.KafkaException;
-
-/**
- *  Thrown if the protocol schema validation fails while parsing request or response.
- */
-public class SchemaException extends KafkaException {
-
-    private static final long serialVersionUID = 1L;
-
-    public SchemaException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Struct.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Struct.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Struct.java
deleted file mode 100644
index c7d8839..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Struct.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.kafka.copied.common.protocol.types;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-/**
- * A record that can be serialized and deserialized according to a pre-defined schema
- */
-public class Struct {
-    private final Schema schema;
-    private final Object[] values;
-
-    Struct(Schema schema, Object[] values) {
-        this.schema = schema;
-        this.values = values;
-    }
-
-    public Struct(Schema schema) {
-        this.schema = schema;
-        this.values = new Object[this.schema.numFields()];
-    }
-
-    /**
-     * The schema for this struct.
-     */
-    public Schema schema() {
-        return this.schema;
-    }
-
-    /**
-     * Return the value of the given pre-validated field, or if the value is missing return the default value.
-     * 
-     * @param field The field for which to get the default value
-     * @throws SchemaException if the field has no value and has no default.
-     */
-    private Object getFieldOrDefault(Field field) {
-        Object value = this.values[field.index];
-        if (value != null)
-            return value;
-        else if (field.defaultValue != Field.NO_DEFAULT)
-            return field.defaultValue;
-        else
-            throw new SchemaException("Missing value for field '" + field.name + "' which has no default value.");
-    }
-
-    /**
-     * Get the value for the field directly by the field index with no lookup needed (faster!)
-     * 
-     * @param field The field to look up
-     * @return The value for that field.
-     * @throws SchemaException if the field has no value and has no default.
-     */
-    public Object get(Field field) {
-        validateField(field);
-        return getFieldOrDefault(field);
-    }
-
-    /**
-     * Get the record value for the field with the given name by doing a hash table lookup (slower!)
-     * 
-     * @param name The name of the field
-     * @return The value in the record
-     * @throws SchemaException If no such field exists
-     */
-    public Object get(String name) {
-        Field field = schema.get(name);
-        if (field == null)
-            throw new SchemaException("No such field: " + name);
-        return getFieldOrDefault(field);
-    }
-
-    /**
-     * Check if the struct contains a field.
-     * @param name
-     * @return Whether a field exists.
-     */
-    public boolean hasField(String name) {
-        return schema.get(name) != null;
-    }
-
-    public Struct getStruct(Field field) {
-        return (Struct) get(field);
-    }
-
-    public Struct getStruct(String name) {
-        return (Struct) get(name);
-    }
-
-    public Short getShort(Field field) {
-        return (Short) get(field);
-    }
-
-    public Short getShort(String name) {
-        return (Short) get(name);
-    }
-
-    public Integer getInt(Field field) {
-        return (Integer) get(field);
-    }
-
-    public Integer getInt(String name) {
-        return (Integer) get(name);
-    }
-
-    public Long getLong(Field field) {
-        return (Long) get(field);
-    }
-
-    public Long getLong(String name) {
-        return (Long) get(name);
-    }
-
-    public Object[] getArray(Field field) {
-        return (Object[]) get(field);
-    }
-
-    public Object[] getArray(String name) {
-        return (Object[]) get(name);
-    }
-
-    public String getString(Field field) {
-        return (String) get(field);
-    }
-
-    public String getString(String name) {
-        return (String) get(name);
-    }
-
-    public ByteBuffer getBytes(Field field) {
-        return (ByteBuffer) get(field);
-    }
-
-    public ByteBuffer getBytes(String name) {
-        return (ByteBuffer) get(name);
-    }
-
-    /**
-     * Set the given field to the specified value
-     * 
-     * @param field The field
-     * @param value The value
-     * @throws SchemaException If the validation of the field failed
-     */
-    public Struct set(Field field, Object value) {
-        validateField(field);
-        this.values[field.index] = value;
-        return this;
-    }
-
-    /**
-     * Set the field specified by the given name to the value
-     * 
-     * @param name The name of the field
-     * @param value The value to set
-     * @throws SchemaException If the field is not known
-     */
-    public Struct set(String name, Object value) {
-        Field field = this.schema.get(name);
-        if (field == null)
-            throw new SchemaException("Unknown field: " + name);
-        this.values[field.index] = value;
-        return this;
-    }
-
-    /**
-     * Create a struct for the schema of a container type (struct or array). Note that for array type, this method
-     * assumes that the type is an array of schema and creates a struct of that schema. Arrays of other types can't be
-     * instantiated with this method.
-     * 
-     * @param field The field to create an instance of
-     * @return The struct
-     * @throws SchemaException If the given field is not a container type
-     */
-    public Struct instance(Field field) {
-        validateField(field);
-        if (field.type() instanceof Schema) {
-            return new Struct((Schema) field.type());
-        } else if (field.type() instanceof ArrayOf) {
-            ArrayOf array = (ArrayOf) field.type();
-            return new Struct((Schema) array.type());
-        } else {
-            throw new SchemaException("Field '" + field.name + "' is not a container type, it is of type " + field.type());
-        }
-    }
-
-    /**
-     * Create a struct instance for the given field which must be a container type (struct or array)
-     * 
-     * @param field The name of the field to create (field must be a schema type)
-     * @return The struct
-     * @throws SchemaException If the given field is not a container type
-     */
-    public Struct instance(String field) {
-        return instance(schema.get(field));
-    }
-
-    /**
-     * Empty all the values from this record
-     */
-    public void clear() {
-        Arrays.fill(this.values, null);
-    }
-
-    /**
-     * Get the serialized size of this object
-     */
-    public int sizeOf() {
-        return this.schema.sizeOf(this);
-    }
-
-    /**
-     * Write this struct to a buffer
-     */
-    public void writeTo(ByteBuffer buffer) {
-        this.schema.write(buffer, this);
-    }
-
-    /**
-     * Ensure the user doesn't try to access fields from the wrong schema
-     *
-     * @throws SchemaException If validation fails
-     */
-    private void validateField(Field field) {
-        if (this.schema != field.schema)
-            throw new SchemaException("Attempt to access field '" + field.name + "' from a different schema instance.");
-        if (field.index > values.length)
-            throw new SchemaException("Invalid field index: " + field.index);
-    }
-
-    /**
-     * Validate the contents of this struct against its schema
-     *
-     * @throws SchemaException If validation fails
-     */
-    public void validate() {
-        this.schema.validate(this);
-    }
-
-    /**
-     * Create a byte buffer containing the serialized form of the values in this struct. This method can choose to break
-     * the struct into multiple ByteBuffers if need be.
-     */
-    public ByteBuffer[] toBytes() {
-        ByteBuffer buffer = ByteBuffer.allocate(sizeOf());
-        writeTo(buffer);
-        return new ByteBuffer[] {buffer};
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder b = new StringBuilder();
-        b.append('{');
-        for (int i = 0; i < this.values.length; i++) {
-            Field f = this.schema.get(i);
-            b.append(f.name);
-            b.append('=');
-            if (f.type() instanceof ArrayOf) {
-                Object[] arrayValue = (Object[]) this.values[i];
-                b.append('[');
-                for (int j = 0; j < arrayValue.length; j++) {
-                    b.append(arrayValue[j]);
-                    if (j < arrayValue.length - 1)
-                        b.append(',');
-                }
-                b.append(']');
-            } else
-                b.append(this.values[i]);
-            if (i < this.values.length - 1)
-                b.append(',');
-        }
-        b.append('}');
-        return b.toString();
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        for (int i = 0; i < this.values.length; i++) {
-            Field f = this.schema.get(i);
-            if (f.type() instanceof ArrayOf) {
-                Object[] arrayObject = (Object[]) this.get(f);
-                for (Object arrayItem: arrayObject)
-                    result = prime * result + arrayItem.hashCode();
-            } else {
-                result = prime * result + this.get(f).hashCode();
-            }
-        }
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        Struct other = (Struct) obj;
-        if (schema != other.schema)
-            return false;
-        for (int i = 0; i < this.values.length; i++) {
-            Field f = this.schema.get(i);
-            Boolean result;
-            if (f.type() instanceof ArrayOf) {
-                result = Arrays.equals((Object[]) this.get(f), (Object[]) other.get(f));
-            } else {
-                result = this.get(f).equals(other.get(f));
-            }
-            if (!result)
-                return false;
-        }
-        return true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Type.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Type.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Type.java
deleted file mode 100644
index acaad74..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/protocol/types/Type.java
+++ /dev/null
@@ -1,250 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.protocol.types;
-
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.nio.ByteBuffer;
-
-/**
- * A serializable type
- */
-public abstract class Type {
-
-    /**
-     * Write the typed object to the buffer
-     *
-     * @throws SchemaException If the object is not valid for its type
-     */
-    public abstract void write(ByteBuffer buffer, Object o);
-
-    /**
-     * Read the typed object from the buffer
-     *
-     * @throws SchemaException If the object is not valid for its type
-     */
-    public abstract Object read(ByteBuffer buffer);
-
-    /**
-     * Validate the object. If succeeded return its typed object.
-     *
-     * @throws SchemaException If validation failed
-     */
-    public abstract Object validate(Object o);
-
-    /**
-     * Return the size of the object in bytes
-     */
-    public abstract int sizeOf(Object o);
-
-    public static final Type INT8 = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            buffer.put((Byte) o);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            return buffer.get();
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 1;
-        }
-
-        @Override
-        public String toString() {
-            return "INT8";
-        }
-
-        @Override
-        public Byte validate(Object item) {
-            if (item instanceof Byte)
-                return (Byte) item;
-            else
-                throw new SchemaException(item + " is not a Byte.");
-        }
-    };
-
-    public static final Type INT16 = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            buffer.putShort((Short) o);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            return buffer.getShort();
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 2;
-        }
-
-        @Override
-        public String toString() {
-            return "INT16";
-        }
-
-        @Override
-        public Short validate(Object item) {
-            if (item instanceof Short)
-                return (Short) item;
-            else
-                throw new SchemaException(item + " is not a Short.");
-        }
-    };
-
-    public static final Type INT32 = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            buffer.putInt((Integer) o);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            return buffer.getInt();
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 4;
-        }
-
-        @Override
-        public String toString() {
-            return "INT32";
-        }
-
-        @Override
-        public Integer validate(Object item) {
-            if (item instanceof Integer)
-                return (Integer) item;
-            else
-                throw new SchemaException(item + " is not an Integer.");
-        }
-    };
-
-    public static final Type INT64 = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            buffer.putLong((Long) o);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            return buffer.getLong();
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 8;
-        }
-
-        @Override
-        public String toString() {
-            return "INT64";
-        }
-
-        @Override
-        public Long validate(Object item) {
-            if (item instanceof Long)
-                return (Long) item;
-            else
-                throw new SchemaException(item + " is not a Long.");
-        }
-    };
-
-    public static final Type STRING = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            byte[] bytes = Utils.utf8((String) o);
-            if (bytes.length > Short.MAX_VALUE)
-                throw new SchemaException("String is longer than the maximum string length.");
-            buffer.putShort((short) bytes.length);
-            buffer.put(bytes);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            int length = buffer.getShort();
-            byte[] bytes = new byte[length];
-            buffer.get(bytes);
-            return Utils.utf8(bytes);
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            return 2 + Utils.utf8Length((String) o);
-        }
-
-        @Override
-        public String toString() {
-            return "STRING";
-        }
-
-        @Override
-        public String validate(Object item) {
-            if (item instanceof String)
-                return (String) item;
-            else
-                throw new SchemaException(item + " is not a String.");
-        }
-    };
-
-    public static final Type BYTES = new Type() {
-        @Override
-        public void write(ByteBuffer buffer, Object o) {
-            ByteBuffer arg = (ByteBuffer) o;
-            int pos = arg.position();
-            buffer.putInt(arg.remaining());
-            buffer.put(arg);
-            arg.position(pos);
-        }
-
-        @Override
-        public Object read(ByteBuffer buffer) {
-            int size = buffer.getInt();
-            ByteBuffer val = buffer.slice();
-            val.limit(size);
-            buffer.position(buffer.position() + size);
-            return val;
-        }
-
-        @Override
-        public int sizeOf(Object o) {
-            ByteBuffer buffer = (ByteBuffer) o;
-            return 4 + buffer.remaining();
-        }
-
-        @Override
-        public String toString() {
-            return "BYTES";
-        }
-
-        @Override
-        public ByteBuffer validate(Object item) {
-            if (item instanceof ByteBuffer)
-                return (ByteBuffer) item;
-            else
-                throw new SchemaException(item + " is not a java.nio.ByteBuffer.");
-        }
-    };
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/ByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/ByteBufferInputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/ByteBufferInputStream.java
deleted file mode 100644
index 4e04cef..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/ByteBufferInputStream.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.kafka.copied.common.record;
-
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-
-/**
- * A byte buffer backed input outputStream
- */
-public class ByteBufferInputStream extends InputStream {
-
-    private ByteBuffer buffer;
-
-    public ByteBufferInputStream(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    public int read() {
-        if (!buffer.hasRemaining()) {
-            return -1;
-        }
-        return buffer.get() & 0xFF;
-    }
-
-    public int read(byte[] bytes, int off, int len) {
-        if (!buffer.hasRemaining()) {
-            return -1;
-        }
-
-        len = Math.min(len, buffer.remaining());
-        buffer.get(bytes, off, len);
-        return len;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/ByteBufferOutputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/ByteBufferOutputStream.java
deleted file mode 100644
index 8b9e189..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/ByteBufferOutputStream.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.kafka.copied.common.record;
-
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-/**
- * A byte buffer backed output outputStream
- */
-public class ByteBufferOutputStream extends OutputStream {
-
-    private static final float REALLOCATION_FACTOR = 1.1f;
-
-    private ByteBuffer buffer;
-
-    public ByteBufferOutputStream(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    public void write(int b) {
-        if (buffer.remaining() < 1)
-            expandBuffer(buffer.capacity() + 1);
-        buffer.put((byte) b);
-    }
-
-    public void write(byte[] bytes, int off, int len) {
-        if (buffer.remaining() < len)
-            expandBuffer(buffer.capacity() + len);
-        buffer.put(bytes, off, len);
-    }
-
-    public ByteBuffer buffer() {
-        return buffer;
-    }
-
-    private void expandBuffer(int size) {
-        int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
-        ByteBuffer temp = ByteBuffer.allocate(expandSize);
-        temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
-        buffer = temp;
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/CompressionType.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/CompressionType.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/CompressionType.java
deleted file mode 100644
index 88f4664..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/CompressionType.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.record;
-
-/**
- * The compression type to use
- */
-public enum CompressionType {
-    NONE(0, "none", 1.0f), GZIP(1, "gzip", 0.5f), SNAPPY(2, "snappy", 0.5f), LZ4(3, "lz4", 0.5f);
-
-    public final int id;
-    public final String name;
-    public final float rate;
-
-    private CompressionType(int id, String name, float rate) {
-        this.id = id;
-        this.name = name;
-        this.rate = rate;
-    }
-
-    public static CompressionType forId(int id) {
-        switch (id) {
-            case 0:
-                return NONE;
-            case 1:
-                return GZIP;
-            case 2:
-                return SNAPPY;
-            case 3:
-                return LZ4;
-            default:
-                throw new IllegalArgumentException("Unknown compression type id: " + id);
-        }
-    }
-
-    public static CompressionType forName(String name) {
-        if (NONE.name.equals(name))
-            return NONE;
-        else if (GZIP.name.equals(name))
-            return GZIP;
-        else if (SNAPPY.name.equals(name))
-            return SNAPPY;
-        else if (LZ4.name.equals(name))
-            return LZ4;
-        else
-            throw new IllegalArgumentException("Unknown compression name: " + name);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Compressor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Compressor.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Compressor.java
deleted file mode 100644
index 49893ab..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/Compressor.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.copied.common.record;
-
-import org.apache.kafka.copied.common.KafkaException;
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.zip.GZIPInputStream;
-import java.util.zip.GZIPOutputStream;
-
-public class Compressor {
-
-    static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f;
-    static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f;
-    static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024;
-
-    private static final float[] TYPE_TO_RATE;
-
-    static {
-        int maxTypeId = -1;
-        for (CompressionType type : CompressionType.values())
-            maxTypeId = Math.max(maxTypeId, type.id);
-        TYPE_TO_RATE = new float[maxTypeId + 1];
-        for (CompressionType type : CompressionType.values()) {
-            TYPE_TO_RATE[type.id] = type.rate;
-        }
-    }
-
-    private final CompressionType type;
-    private final DataOutputStream appendStream;
-    private final ByteBufferOutputStream bufferStream;
-    private final int initPos;
-
-    public long writtenUncompressed;
-    public long numRecords;
-
-    public Compressor(ByteBuffer buffer, CompressionType type, int blockSize) {
-        this.type = type;
-        this.initPos = buffer.position();
-
-        this.numRecords = 0;
-        this.writtenUncompressed = 0;
-
-        if (type != CompressionType.NONE) {
-            // for compressed records, leave space for the header and the shallow message metadata
-            // and move the starting position to the value payload offset
-            buffer.position(initPos + Records.LOG_OVERHEAD + Record.RECORD_OVERHEAD);
-        }
-
-        // create the stream
-        bufferStream = new ByteBufferOutputStream(buffer);
-        appendStream = wrapForOutput(bufferStream, type, blockSize);
-    }
-
-    public Compressor(ByteBuffer buffer, CompressionType type) {
-        this(buffer, type, COMPRESSION_DEFAULT_BUFFER_SIZE);
-    }
-
-    public ByteBuffer buffer() {
-        return bufferStream.buffer();
-    }
-    
-    public double compressionRate() {
-        ByteBuffer buffer = bufferStream.buffer();
-        if (this.writtenUncompressed == 0)
-            return 1.0;
-        else
-            return (double) buffer.position() / this.writtenUncompressed;
-    }
-
-    public void close() {
-        try {
-            appendStream.close();
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-
-        if (type != CompressionType.NONE) {
-            ByteBuffer buffer = bufferStream.buffer();
-            int pos = buffer.position();
-            // write the header, for the end offset write as number of records - 1
-            buffer.position(initPos);
-            buffer.putLong(numRecords - 1);
-            buffer.putInt(pos - initPos - Records.LOG_OVERHEAD);
-            // write the shallow message (the crc and value size are not correct yet)
-            Record.write(buffer, null, null, type, 0, -1);
-            // compute the fill the value size
-            int valueSize = pos - initPos - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD;
-            buffer.putInt(initPos + Records.LOG_OVERHEAD + Record.KEY_OFFSET, valueSize);
-            // compute and fill the crc at the beginning of the message
-            long crc = Record.computeChecksum(buffer,
-                    initPos + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET,
-                    pos - initPos - Records.LOG_OVERHEAD - Record.MAGIC_OFFSET);
-            Utils.writeUnsignedInt(buffer, initPos + Records.LOG_OVERHEAD + Record.CRC_OFFSET, crc);
-            // reset the position
-            buffer.position(pos);
-
-            // update the compression ratio
-            float compressionRate = (float) buffer.position() / this.writtenUncompressed;
-            TYPE_TO_RATE[type.id] = TYPE_TO_RATE[type.id] * COMPRESSION_RATE_DAMPING_FACTOR +
-                compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR);
-        }
-    }
-
-    // Note that for all the write operations below, IO exceptions should
-    // never be thrown since the underlying ByteBufferOutputStream does not throw IOException;
-    // therefore upon encountering this issue we just close the append stream.
-
-    public void putLong(final long value) {
-        try {
-            appendStream.writeLong(value);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void putInt(final int value) {
-        try {
-            appendStream.writeInt(value);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void put(final ByteBuffer buffer) {
-        try {
-            appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit());
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void putByte(final byte value) {
-        try {
-            appendStream.write(value);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void put(final byte[] bytes, final int offset, final int len) {
-        try {
-            appendStream.write(bytes, offset, len);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void putRecord(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        // put a record as un-compressed into the underlying stream
-        long crc = Record.computeChecksum(key, value, type, valueOffset, valueSize);
-        byte attributes = Record.computeAttributes(type);
-        putRecord(crc, attributes, key, value, valueOffset, valueSize);
-    }
-
-    public void putRecord(byte[] key, byte[] value) {
-        putRecord(key, value, CompressionType.NONE, 0, -1);
-    }
-
-    private void putRecord(final long crc, final byte attributes, final byte[] key, final byte[] value, final int valueOffset, final int valueSize) {
-        Record.write(this, crc, attributes, key, value, valueOffset, valueSize);
-    }
-
-    public void recordWritten(int size) {
-        numRecords += 1;
-        writtenUncompressed += size;
-    }
-
-    public long estimatedBytesWritten() {
-        if (type == CompressionType.NONE) {
-            return bufferStream.buffer().position();
-        } else {
-            // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
-            return (long) (writtenUncompressed * TYPE_TO_RATE[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR);
-        }
-    }
-
-    // the following two functions also need to be public since they are used in MemoryRecords.iteration
-
-    static public DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, int bufferSize) {
-        try {
-            switch (type) {
-                case NONE:
-                    return new DataOutputStream(buffer);
-                case GZIP:
-                    return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize));
-                case SNAPPY:
-                    // dynamically load the snappy class to avoid runtime dependency
-                    // on snappy if we are not using it
-                    try {
-                        Class<?> outputStreamClass = Class.forName("org.xerial.snappy.SnappyOutputStream");
-                        OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class, Integer.TYPE)
-                            .newInstance(buffer, bufferSize);
-                        return new DataOutputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                case LZ4:
-                    try {
-                        Class<?> outputStreamClass = Class.forName("org.apache.KafkaLZ4BlockOutputStream");
-                        OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class)
-                            .newInstance(buffer);
-                        return new DataOutputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                default:
-                    throw new IllegalArgumentException("Unknown compression type: " + type);
-            }
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-    }
-
-    static public DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type) {
-        try {
-            switch (type) {
-                case NONE:
-                    return new DataInputStream(buffer);
-                case GZIP:
-                    return new DataInputStream(new GZIPInputStream(buffer));
-                case SNAPPY:
-                    // dynamically load the snappy class to avoid runtime dependency
-                    // on snappy if we are not using it
-                    try {
-                        Class<?> inputStreamClass = Class.forName("org.xerial.snappy.SnappyInputStream");
-                        InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class)
-                            .newInstance(buffer);
-                        return new DataInputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                case LZ4:
-                    // dynamically load LZ4 class to avoid runtime dependency
-                    try {
-                        Class<?> inputStreamClass = Class.forName("org.apache.KafkaLZ4BlockInputStream");
-                        InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class)
-                            .newInstance(buffer);
-                        return new DataInputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                default:
-                    throw new IllegalArgumentException("Unknown compression type: " + type);
-            }
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-    }
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/KafkaLZ4BlockInputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/KafkaLZ4BlockInputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/KafkaLZ4BlockInputStream.java
deleted file mode 100644
index 22c5e13..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/KafkaLZ4BlockInputStream.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.kafka.copied.common.record;
-
-import net.jpountz.lz4.LZ4Exception;
-import net.jpountz.lz4.LZ4Factory;
-import net.jpountz.lz4.LZ4SafeDecompressor;
-import net.jpountz.xxhash.XXHash32;
-import net.jpountz.xxhash.XXHashFactory;
-import org.apache.kafka.copied.common.record.KafkaLZ4BlockOutputStream.BD;
-import org.apache.kafka.copied.common.record.KafkaLZ4BlockOutputStream.FLG;
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.io.FilterInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-import static org.apache.kafka.copied.common.record.KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK;
-import static org.apache.kafka.copied.common.record.KafkaLZ4BlockOutputStream.LZ4_MAX_HEADER_LENGTH;
-import static org.apache.kafka.copied.common.record.KafkaLZ4BlockOutputStream.MAGIC;
-
-/**
- * A partial implementation of the v1.4.1 LZ4 Frame format.
- * 
- * @see <a href="https://docs.google.com/document/d/1Tdxmn5_2e5p1y4PtXkatLndWVb0R8QARJFe6JI4Keuo/edit">LZ4 Framing
- *      Format Spec</a>
- */
-public final class KafkaLZ4BlockInputStream extends FilterInputStream {
-
-    public static final String PREMATURE_EOS = "Stream ended prematurely";
-    public static final String NOT_SUPPORTED = "Stream unsupported";
-    public static final String BLOCK_HASH_MISMATCH = "Block checksum mismatch";
-    public static final String DESCRIPTOR_HASH_MISMATCH = "Stream frame descriptor corrupted";
-
-    private final LZ4SafeDecompressor decompressor;
-    private final XXHash32 checksum;
-    private final byte[] buffer;
-    private final byte[] compressedBuffer;
-    private final int maxBlockSize;
-    private FLG flg;
-    private BD bd;
-    private int bufferOffset;
-    private int bufferSize;
-    private boolean finished;
-
-    /**
-     * Create a new {@link InputStream} that will decompress data using the LZ4 algorithm.
-     * 
-     * @param in The stream to decompress
-     * @throws IOException
-     */
-    public KafkaLZ4BlockInputStream(InputStream in) throws IOException {
-        super(in);
-        decompressor = LZ4Factory.fastestInstance().safeDecompressor();
-        checksum = XXHashFactory.fastestInstance().hash32();
-        readHeader();
-        maxBlockSize = bd.getBlockMaximumSize();
-        buffer = new byte[maxBlockSize];
-        compressedBuffer = new byte[maxBlockSize];
-        bufferOffset = 0;
-        bufferSize = 0;
-        finished = false;
-    }
-
-    /**
-     * Reads the magic number and frame descriptor from the underlying {@link InputStream}.
-     * 
-     * @throws IOException
-     */
-    private void readHeader() throws IOException {
-        byte[] header = new byte[LZ4_MAX_HEADER_LENGTH];
-
-        // read first 6 bytes into buffer to check magic and FLG/BD descriptor flags
-        bufferOffset = 6;
-        if (in.read(header, 0, bufferOffset) != bufferOffset) {
-            throw new IOException(PREMATURE_EOS);
-        }
-
-        if (MAGIC != Utils.readUnsignedIntLE(header, bufferOffset - 6)) {
-            throw new IOException(NOT_SUPPORTED);
-        }
-        flg = FLG.fromByte(header[bufferOffset - 2]);
-        bd = BD.fromByte(header[bufferOffset - 1]);
-        // TODO read uncompressed content size, update flg.validate()
-        // TODO read dictionary id, update flg.validate()
-
-        // check stream descriptor hash
-        byte hash = (byte) ((checksum.hash(header, 0, bufferOffset, 0) >> 8) & 0xFF);
-        header[bufferOffset++] = (byte) in.read();
-        if (hash != header[bufferOffset - 1]) {
-            throw new IOException(DESCRIPTOR_HASH_MISMATCH);
-        }
-    }
-
-    /**
-     * Decompresses (if necessary) buffered data, optionally computes and validates a XXHash32 checksum, and writes the
-     * result to a buffer.
-     * 
-     * @throws IOException
-     */
-    private void readBlock() throws IOException {
-        int blockSize = Utils.readUnsignedIntLE(in);
-
-        // Check for EndMark
-        if (blockSize == 0) {
-            finished = true;
-            // TODO implement content checksum, update flg.validate()
-            return;
-        } else if (blockSize > maxBlockSize) {
-            throw new IOException(String.format("Block size %s exceeded max: %s", blockSize, maxBlockSize));
-        }
-
-        boolean compressed = (blockSize & LZ4_FRAME_INCOMPRESSIBLE_MASK) == 0;
-        byte[] bufferToRead;
-        if (compressed) {
-            bufferToRead = compressedBuffer;
-        } else {
-            blockSize &= ~LZ4_FRAME_INCOMPRESSIBLE_MASK;
-            bufferToRead = buffer;
-            bufferSize = blockSize;
-        }
-
-        if (in.read(bufferToRead, 0, blockSize) != blockSize) {
-            throw new IOException(PREMATURE_EOS);
-        }
-
-        // verify checksum
-        if (flg.isBlockChecksumSet() && Utils.readUnsignedIntLE(in) != checksum.hash(bufferToRead, 0, blockSize, 0)) {
-            throw new IOException(BLOCK_HASH_MISMATCH);
-        }
-
-        if (compressed) {
-            try {
-                bufferSize = decompressor.decompress(compressedBuffer, 0, blockSize, buffer, 0, maxBlockSize);
-            } catch (LZ4Exception e) {
-                throw new IOException(e);
-            }
-        }
-
-        bufferOffset = 0;
-    }
-
-    @Override
-    public int read() throws IOException {
-        if (finished) {
-            return -1;
-        }
-        if (available() == 0) {
-            readBlock();
-        }
-        if (finished) {
-            return -1;
-        }
-        int value = buffer[bufferOffset++] & 0xFF;
-
-        return value;
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        net.jpountz.util.Utils.checkRange(b, off, len);
-        if (finished) {
-            return -1;
-        }
-        if (available() == 0) {
-            readBlock();
-        }
-        if (finished) {
-            return -1;
-        }
-        len = Math.min(len, available());
-        System.arraycopy(buffer, bufferOffset, b, off, len);
-        bufferOffset += len;
-        return len;
-    }
-
-    @Override
-    public long skip(long n) throws IOException {
-        if (finished) {
-            return 0;
-        }
-        if (available() == 0) {
-            readBlock();
-        }
-        if (finished) {
-            return 0;
-        }
-        n = Math.min(n, available());
-        bufferOffset += n;
-        return n;
-    }
-
-    @Override
-    public int available() throws IOException {
-        return bufferSize - bufferOffset;
-    }
-
-    @Override
-    public void close() throws IOException {
-        in.close();
-    }
-
-    @Override
-    public synchronized void mark(int readlimit) {
-        throw new RuntimeException("mark not supported");
-    }
-
-    @Override
-    public synchronized void reset() throws IOException {
-        throw new RuntimeException("reset not supported");
-    }
-
-    @Override
-    public boolean markSupported() {
-        return false;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/KafkaLZ4BlockOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/KafkaLZ4BlockOutputStream.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/KafkaLZ4BlockOutputStream.java
deleted file mode 100644
index b9cfb5a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/KafkaLZ4BlockOutputStream.java
+++ /dev/null
@@ -1,391 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * 
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.copied.common.record;
-
-import net.jpountz.lz4.LZ4Compressor;
-import net.jpountz.lz4.LZ4Factory;
-import net.jpountz.xxhash.XXHash32;
-import net.jpountz.xxhash.XXHashFactory;
-import org.apache.kafka.copied.common.utils.Utils;
-
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * A partial implementation of the v1.4.1 LZ4 Frame format.
- * 
- * @see <a href="https://docs.google.com/document/d/1Tdxmn5_2e5p1y4PtXkatLndWVb0R8QARJFe6JI4Keuo/edit">LZ4 Framing
- *      Format Spec</a>
- */
-public final class KafkaLZ4BlockOutputStream extends FilterOutputStream {
-
-    public static final int MAGIC = 0x184D2204;
-    public static final int LZ4_MAX_HEADER_LENGTH = 19;
-    public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000;
-
-    public static final String CLOSED_STREAM = "The stream is already closed";
-
-    public static final int BLOCKSIZE_64KB = 4;
-    public static final int BLOCKSIZE_256KB = 5;
-    public static final int BLOCKSIZE_1MB = 6;
-    public static final int BLOCKSIZE_4MB = 7;
-
-    private final LZ4Compressor compressor;
-    private final XXHash32 checksum;
-    private final FLG flg;
-    private final BD bd;
-    private final byte[] buffer;
-    private final byte[] compressedBuffer;
-    private final int maxBlockSize;
-    private int bufferOffset;
-    private boolean finished;
-
-    /**
-     * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
-     * 
-     * @param out The output stream to compress
-     * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other
-     *            values will generate an exception
-     * @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for
-     *            every block of data
-     * @throws IOException
-     */
-    public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum) throws IOException {
-        super(out);
-        compressor = LZ4Factory.fastestInstance().fastCompressor();
-        checksum = XXHashFactory.fastestInstance().hash32();
-        bd = new BD(blockSize);
-        flg = new FLG(blockChecksum);
-        bufferOffset = 0;
-        maxBlockSize = bd.getBlockMaximumSize();
-        buffer = new byte[maxBlockSize];
-        compressedBuffer = new byte[compressor.maxCompressedLength(maxBlockSize)];
-        finished = false;
-        writeHeader();
-    }
-
-    /**
-     * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
-     * 
-     * @param out The stream to compress
-     * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other
-     *            values will generate an exception
-     * @throws IOException
-     */
-    public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize) throws IOException {
-        this(out, blockSize, false);
-    }
-
-    /**
-     * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
-     * 
-     * @param out The output stream to compress
-     * @throws IOException
-     */
-    public KafkaLZ4BlockOutputStream(OutputStream out) throws IOException {
-        this(out, BLOCKSIZE_64KB);
-    }
-
-    /**
-     * Writes the magic number and frame descriptor to the underlying {@link OutputStream}.
-     * 
-     * @throws IOException
-     */
-    private void writeHeader() throws IOException {
-        Utils.writeUnsignedIntLE(buffer, 0, MAGIC);
-        bufferOffset = 4;
-        buffer[bufferOffset++] = flg.toByte();
-        buffer[bufferOffset++] = bd.toByte();
-        // TODO write uncompressed content size, update flg.validate()
-        // TODO write dictionary id, update flg.validate()
-        // compute checksum on all descriptor fields
-        int hash = (checksum.hash(buffer, 0, bufferOffset, 0) >> 8) & 0xFF;
-        buffer[bufferOffset++] = (byte) hash;
-        // write out frame descriptor
-        out.write(buffer, 0, bufferOffset);
-        bufferOffset = 0;
-    }
-
-    /**
-     * Compresses buffered data, optionally computes an XXHash32 checksum, and writes the result to the underlying
-     * {@link OutputStream}.
-     * 
-     * @throws IOException
-     */
-    private void writeBlock() throws IOException {
-        if (bufferOffset == 0) {
-            return;
-        }
-
-        int compressedLength = compressor.compress(buffer, 0, bufferOffset, compressedBuffer, 0);
-        byte[] bufferToWrite = compressedBuffer;
-        int compressMethod = 0;
-
-        // Store block uncompressed if compressed length is greater (incompressible)
-        if (compressedLength >= bufferOffset) {
-            bufferToWrite = buffer;
-            compressedLength = bufferOffset;
-            compressMethod = LZ4_FRAME_INCOMPRESSIBLE_MASK;
-        }
-
-        // Write content
-        Utils.writeUnsignedIntLE(out, compressedLength | compressMethod);
-        out.write(bufferToWrite, 0, compressedLength);
-
-        // Calculate and write block checksum
-        if (flg.isBlockChecksumSet()) {
-            int hash = checksum.hash(bufferToWrite, 0, compressedLength, 0);
-            Utils.writeUnsignedIntLE(out, hash);
-        }
-        bufferOffset = 0;
-    }
-
-    /**
-     * Similar to the {@link #writeBlock()} method. Writes a 0-length block (without block checksum) to signal the end
-     * of the block stream.
-     * 
-     * @throws IOException
-     */
-    private void writeEndMark() throws IOException {
-        Utils.writeUnsignedIntLE(out, 0);
-        // TODO implement content checksum, update flg.validate()
-        finished = true;
-    }
-
-    @Override
-    public void write(int b) throws IOException {
-        ensureNotFinished();
-        if (bufferOffset == maxBlockSize) {
-            writeBlock();
-        }
-        buffer[bufferOffset++] = (byte) b;
-    }
-
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-        net.jpountz.util.Utils.checkRange(b, off, len);
-        ensureNotFinished();
-
-        int bufferRemainingLength = maxBlockSize - bufferOffset;
-        // while b will fill the buffer
-        while (len > bufferRemainingLength) {
-            // fill remaining space in buffer
-            System.arraycopy(b, off, buffer, bufferOffset, bufferRemainingLength);
-            bufferOffset = maxBlockSize;
-            writeBlock();
-            // compute new offset and length
-            off += bufferRemainingLength;
-            len -= bufferRemainingLength;
-            bufferRemainingLength = maxBlockSize;
-        }
-
-        System.arraycopy(b, off, buffer, bufferOffset, len);
-        bufferOffset += len;
-    }
-
-    @Override
-    public void flush() throws IOException {
-        if (!finished) {
-            writeBlock();
-        }
-        if (out != null) {
-            out.flush();
-        }
-    }
-
-    /**
-     * A simple state check to ensure the stream is still open.
-     */
-    private void ensureNotFinished() {
-        if (finished) {
-            throw new IllegalStateException(CLOSED_STREAM);
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (!finished) {
-            writeEndMark();
-            flush();
-            finished = true;
-        }
-        if (out != null) {
-            out.close();
-            out = null;
-        }
-    }
-
-    public static class FLG {
-
-        private static final int VERSION = 1;
-
-        private final int presetDictionary;
-        private final int reserved1;
-        private final int contentChecksum;
-        private final int contentSize;
-        private final int blockChecksum;
-        private final int blockIndependence;
-        private final int version;
-
-        public FLG() {
-            this(false);
-        }
-
-        public FLG(boolean blockChecksum) {
-            this(0, 0, 0, 0, blockChecksum ? 1 : 0, 1, VERSION);
-        }
-
-        private FLG(int presetDictionary,
-                    int reserved1,
-                    int contentChecksum,
-                    int contentSize,
-                    int blockChecksum,
-                    int blockIndependence,
-                    int version) {
-            this.presetDictionary = presetDictionary;
-            this.reserved1 = reserved1;
-            this.contentChecksum = contentChecksum;
-            this.contentSize = contentSize;
-            this.blockChecksum = blockChecksum;
-            this.blockIndependence = blockIndependence;
-            this.version = version;
-            validate();
-        }
-
-        public static FLG fromByte(byte flg) {
-            int presetDictionary = (flg >>> 0) & 1;
-            int reserved1 = (flg >>> 1) & 1;
-            int contentChecksum = (flg >>> 2) & 1;
-            int contentSize = (flg >>> 3) & 1;
-            int blockChecksum = (flg >>> 4) & 1;
-            int blockIndependence = (flg >>> 5) & 1;
-            int version = (flg >>> 6) & 3;
-
-            return new FLG(presetDictionary,
-                           reserved1,
-                           contentChecksum,
-                           contentSize,
-                           blockChecksum,
-                           blockIndependence,
-                           version);
-        }
-
-        public byte toByte() {
-            return (byte) (((presetDictionary & 1) << 0) | ((reserved1 & 1) << 1) | ((contentChecksum & 1) << 2)
-                    | ((contentSize & 1) << 3) | ((blockChecksum & 1) << 4) | ((blockIndependence & 1) << 5) | ((version & 3) << 6));
-        }
-
-        private void validate() {
-            if (presetDictionary != 0) {
-                throw new RuntimeException("Preset dictionary is unsupported");
-            }
-            if (reserved1 != 0) {
-                throw new RuntimeException("Reserved1 field must be 0");
-            }
-            if (contentChecksum != 0) {
-                throw new RuntimeException("Content checksum is unsupported");
-            }
-            if (contentSize != 0) {
-                throw new RuntimeException("Content size is unsupported");
-            }
-            if (blockIndependence != 1) {
-                throw new RuntimeException("Dependent block stream is unsupported");
-            }
-            if (version != VERSION) {
-                throw new RuntimeException(String.format("Version %d is unsupported", version));
-            }
-        }
-
-        public boolean isPresetDictionarySet() {
-            return presetDictionary == 1;
-        }
-
-        public boolean isContentChecksumSet() {
-            return contentChecksum == 1;
-        }
-
-        public boolean isContentSizeSet() {
-            return contentSize == 1;
-        }
-
-        public boolean isBlockChecksumSet() {
-            return blockChecksum == 1;
-        }
-
-        public boolean isBlockIndependenceSet() {
-            return blockIndependence == 1;
-        }
-
-        public int getVersion() {
-            return version;
-        }
-    }
-
-    public static class BD {
-
-        private final int reserved2;
-        private final int blockSizeValue;
-        private final int reserved3;
-
-        public BD() {
-            this(0, BLOCKSIZE_64KB, 0);
-        }
-
-        public BD(int blockSizeValue) {
-            this(0, blockSizeValue, 0);
-        }
-
-        private BD(int reserved2, int blockSizeValue, int reserved3) {
-            this.reserved2 = reserved2;
-            this.blockSizeValue = blockSizeValue;
-            this.reserved3 = reserved3;
-            validate();
-        }
-
-        public static BD fromByte(byte bd) {
-            int reserved2 = (bd >>> 0) & 15;
-            int blockMaximumSize = (bd >>> 4) & 7;
-            int reserved3 = (bd >>> 7) & 1;
-
-            return new BD(reserved2, blockMaximumSize, reserved3);
-        }
-
-        private void validate() {
-            if (reserved2 != 0) {
-                throw new RuntimeException("Reserved2 field must be 0");
-            }
-            if (blockSizeValue < 4 || blockSizeValue > 7) {
-                throw new RuntimeException("Block size value must be between 4 and 7");
-            }
-            if (reserved3 != 0) {
-                throw new RuntimeException("Reserved3 field must be 0");
-            }
-        }
-
-        // 2^(2n+8)
-        public int getBlockMaximumSize() {
-            return 1 << ((2 * blockSizeValue) + 8);
-        }
-
-        public byte toByte() {
-            return (byte) (((reserved2 & 15) << 0) | ((blockSizeValue & 7) << 4) | ((reserved3 & 1) << 7));
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/33f4c818/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/LogEntry.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/LogEntry.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/LogEntry.java
deleted file mode 100644
index 2190936..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka-083/src/main/java/org/apache/kafka/copied/common/record/LogEntry.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.kafka.copied.common.record;
-
-/**
- * An offset and record pair
- */
-public final class LogEntry {
-
-    private final long offset;
-    private final Record record;
-
-    public LogEntry(long offset, Record record) {
-        this.offset = offset;
-        this.record = record;
-    }
-
-    public long offset() {
-        return this.offset;
-    }
-
-    public Record record() {
-        return this.record;
-    }
-
-    @Override
-    public String toString() {
-        return "LogEntry(" + offset + ", " + record + ")";
-    }
-    
-    public int size() {
-        return record.size() + Records.LOG_OVERHEAD;
-    }
-}